diff --git a/.asf.yaml b/.asf.yaml index 76429ad286de..dafb78888389 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -27,6 +27,9 @@ github: - spark-sql - vectorization - velox + collaborators: + - majetideepak + - pedroerp enabled_merge_buttons: squash: true merge: false diff --git a/.github/labeler.yml b/.github/labeler.yml index abc62156f40f..5deb2714ca6f 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -60,7 +60,7 @@ CORE: VELOX: - changed-files: - any-glob-to-any-file: [ - 'gluten-data/**/*', + 'gluten-arrow/**/*', 'backends-velox/**/*', 'ep/build-velox/**/*', 'cpp/**/*' diff --git a/.github/workflows/build_bundle_package.yml b/.github/workflows/build_bundle_package.yml index 3afe5b5e02a8..c7ddd8748b25 100644 --- a/.github/workflows/build_bundle_package.yml +++ b/.github/workflows/build_bundle_package.yml @@ -17,6 +17,7 @@ name: Build bundle package env: ACTIONS_ALLOW_USE_UNSECURE_NODE_VERSION: true + CCACHE_DIR: "${{ github.workspace }}/.ccache" concurrency: group: ${{ github.repository }}-${{ github.head_ref || github.sha }}-${{ github.workflow }} @@ -25,10 +26,6 @@ concurrency: on: workflow_dispatch: inputs: - os: - description: 'OS version: ubuntu:20.04, ubuntu:22.04, centos:7 or centos:8' - required: true - default: 'ubuntu:20.04' spark: description: 'Spark version: spark-3.2, spark-3.3, spark-3.4 or spark-3.5' required: true @@ -41,123 +38,58 @@ on: jobs: build-native-lib: runs-on: ubuntu-20.04 - container: apache/gluten:gluten-vcpkg-builder_2024_05_29 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 + - name: Get Ccache + uses: actions/cache/restore@v3 + with: + path: '${{ env.CCACHE_DIR }}' + key: ccache-centos7-release-default-${{github.sha}} + restore-keys: | + ccache-centos7-release-default - name: Build Gluten velox third party run: | - yum install sudo patch java-1.8.0-openjdk-devel -y && \ - cd $GITHUB_WORKSPACE/ep/build-velox/src && \ - ./get_velox.sh && \ - source /opt/rh/devtoolset-11/enable && \ - cd $GITHUB_WORKSPACE/ && \ - export NUM_THREADS=4 - ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_tests=OFF --build_benchmarks=OFF --enable_s3=OFF \ - --enable_gcs=OFF --enable_hdfs=ON --enable_abfs=OFF + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " - name: Upload native libs - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: ./cpp/build/releases/ name: velox-native-lib-${{github.sha}} retention-days: 1 - name: Upload Artifact Arrow Jar - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: path: /root/.m2/repository/org/apache/arrow/ name: velox-arrow-jar-centos-7-${{github.sha}} - build-bundle-package-ubuntu: - if: startsWith(github.event.inputs.os, 'ubuntu') - needs: build-native-lib - runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} - steps: - - uses: actions/checkout@v2 - - name: Download All Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-native-lib-${{github.sha}} - path: ./cpp/build/releases - - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-arrow-jar-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ - - name: Setup java and maven - run: | - apt-get update && \ - apt-get install -y openjdk-8-jdk maven && \ - apt remove openjdk-11* -y - - name: Build for Spark ${{ github.event.inputs.spark }} - run: | - cd $GITHUB_WORKSPACE/ && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip - - name: Upload bundle package - uses: actions/upload-artifact@v2 - with: - name: gluten-velox-bundle-package - path: package/target/gluten-velox-bundle-*.jar - retention-days: 7 - - build-bundle-package-centos7: - if: ${{ github.event.inputs.os == 'centos:7' }} - needs: build-native-lib - runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} - steps: - - uses: actions/checkout@v2 - - name: Download All Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-native-lib-${{github.sha}} - path: ./cpp/build/releases - - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 - with: - name: velox-arrow-jar-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ - - name: Setup java and maven - run: | - yum update -y && yum install -y java-1.8.0-openjdk-devel wget && \ - wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz && \ - tar -xvf apache-maven-3.8.8-bin.tar.gz && \ - mv apache-maven-3.8.8 /usr/lib/maven - - name: Build for Spark ${{ github.event.inputs.spark }} - run: | - cd $GITHUB_WORKSPACE/ && \ - export MAVEN_HOME=/usr/lib/maven && \ - export PATH=${PATH}:${MAVEN_HOME}/bin && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip - - name: Upload bundle package - uses: actions/upload-artifact@v2 - with: - name: gluten-velox-bundle-package - path: package/target/gluten-velox-bundle-*.jar - retention-days: 7 - build-bundle-package-centos8: - if: ${{ github.event.inputs.os == 'centos:8' }} needs: build-native-lib runs-on: ubuntu-20.04 - container: ${{ github.event.inputs.os }} + container: centos:8 steps: - uses: actions/checkout@v2 - name: Download All Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-native-lib-${{github.sha}} path: ./cpp/build/releases - name: Download All Arrow Jar Artifacts - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: velox-arrow-jar-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true && \ - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - name: Setup java and maven run: | + sed -i 's/mirrorlist/#mirrorlist/g' /etc/yum.repos.d/CentOS-* && \ + sed -i 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* && \ yum update -y && yum install -y java-1.8.0-openjdk-devel wget && \ wget https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz && \ tar -xvf apache-maven-3.8.8-bin.tar.gz && \ @@ -167,10 +99,11 @@ jobs: cd $GITHUB_WORKSPACE/ && \ export MAVEN_HOME=/usr/lib/maven && \ export PATH=${PATH}:${MAVEN_HOME}/bin && \ - mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -DskipTests -Dmaven.source.skip + mvn clean install -P${{ github.event.inputs.spark }} -Dhadoop.version=${{ github.event.inputs.hadoop }} -Pbackends-velox -Pceleborn -Puniffle -DskipTests -Dmaven.source.skip - name: Upload bundle package - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: gluten-velox-bundle-package path: package/target/gluten-velox-bundle-*.jar retention-days: 7 + diff --git a/.github/workflows/clickhouse_be_trigger.yml b/.github/workflows/clickhouse_be_trigger.yml index 153fa391b8de..a45dfc8a948d 100644 --- a/.github/workflows/clickhouse_be_trigger.yml +++ b/.github/workflows/clickhouse_be_trigger.yml @@ -47,9 +47,16 @@ jobs: with: github-token: ${{ secrets.GITHUB_TOKEN }} script: | + const issueNumber = context.payload.number; + let body; + if (issueNumber % 10 === 0) { + body = "Run Gluten ClickHouse CI on ARM"; + } else { + body = "Run Gluten Clickhouse CI on x86"; + } await github.rest.issues.createComment({ owner: context.repo.owner, repo: context.repo.repo, - issue_number: context.payload.number, - body: "Run Gluten Clickhouse CI" + issue_number: issueNumber, + body: body }); diff --git a/.github/workflows/util/install_spark_resources.sh b/.github/workflows/util/install_spark_resources.sh index b61414166b90..699efd6c3de2 100755 --- a/.github/workflows/util/install_spark_resources.sh +++ b/.github/workflows/util/install_spark_resources.sh @@ -63,26 +63,26 @@ case "$1" in 3.5) # Spark-3.5 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.12 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; 3.5-scala2.13) # Spark-3.5, scala 2.13 cd ${INSTALL_DIR} && \ - wget -nv https://archive.apache.org/dist/spark/spark-3.5.3/spark-3.5.3-bin-hadoop3.tgz && \ - tar --strip-components=1 -xf spark-3.5.3-bin-hadoop3.tgz spark-3.5.3-bin-hadoop3/jars/ && \ - rm -rf spark-3.5.3-bin-hadoop3.tgz && \ + wget -nv https://archive.apache.org/dist/spark/spark-3.5.2/spark-3.5.2-bin-hadoop3.tgz && \ + tar --strip-components=1 -xf spark-3.5.2-bin-hadoop3.tgz spark-3.5.2-bin-hadoop3/jars/ && \ + rm -rf spark-3.5.2-bin-hadoop3.tgz && \ mkdir -p ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ mv jars ${INSTALL_DIR}/shims/spark35/spark_home/assembly/target/scala-2.13 && \ - wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.3.tar.gz && \ - tar --strip-components=1 -xf v3.5.3.tar.gz spark-3.5.3/sql/core/src/test/resources/ && \ + wget -nv https://github.com/apache/spark/archive/refs/tags/v3.5.2.tar.gz && \ + tar --strip-components=1 -xf v3.5.2.tar.gz spark-3.5.2/sql/core/src/test/resources/ && \ mkdir -p shims/spark35/spark_home/ && \ mv sql shims/spark35/spark_home/ ;; diff --git a/.github/workflows/velox_backend.yml b/.github/workflows/velox_backend.yml index 2fa0bb576a6f..c790540367e1 100644 --- a/.github/workflows/velox_backend.yml +++ b/.github/workflows/velox_backend.yml @@ -56,9 +56,8 @@ concurrency: jobs: build-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -68,10 +67,18 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - yum install ccache -y - cd $GITHUB_WORKSPACE/ - bash dev/ci-velox-buildstatic-centos-7.sh + docker pull apache/gluten:vcpkg-centos-7 + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + df -a + cd /work + export CCACHE_DIR=/work/.ccache + mkdir -p /work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + ccache -s + mkdir -p /work/.m2/repository/org/apache/arrow/ + cp -r /root/.m2/repository/org/apache/arrow/* /work/.m2/repository/org/apache/arrow/ + " + - name: "Save ccache" uses: actions/cache/save@v3 id: ccache @@ -85,7 +92,7 @@ jobs: - uses: actions/upload-artifact@v3 with: name: arrow-jars-centos-7-${{github.sha}} - path: /root/.m2/repository/org/apache/arrow/ + path: .m2/repository/org/apache/arrow/ run-tpc-test-ubuntu: needs: build-native-lib-centos-7 @@ -158,12 +165,12 @@ jobs: && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 - run-tpc-test-centos: + run-tpc-test-centos8: needs: build-native-lib-centos-7 strategy: fail-fast: false matrix: - os: [ "centos:7", "centos:8" ] + os: [ "centos:8" ] spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] java: [ "java-8", "java-11", "java-17" ] # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 @@ -249,15 +256,17 @@ jobs: --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ --extra-conf=spark.gluten.ras.enabled=true - run-tpc-test-ubuntu-iothreads: + run-tpc-test-centos7: needs: build-native-lib-centos-7 strategy: fail-fast: false matrix: - spark: [ "spark-3.5" ] + spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5" ] + java: [ "java-8" ] + # Spark supports JDK17 since 3.3 and later, see https://issues.apache.org/jira/browse/SPARK-33772 runs-on: ubuntu-20.04 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Download All Native Artifacts uses: actions/download-artifact@v3 with: @@ -267,29 +276,47 @@ jobs: uses: actions/download-artifact@v3 with: name: arrow-jars-centos-7-${{github.sha}} - path: /home/runner/.m2/repository/org/apache/arrow/ - - name: Setup java and maven - run: | - sudo apt-get update - sudo apt-get install -y openjdk-8-jdk maven - - name: Set environment variables - run: | - echo "JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64" >> $GITHUB_ENV - - name: Build for Spark ${{ matrix.spark }} - run: | - cd $GITHUB_WORKSPACE/ - $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -DskipTests - cd $GITHUB_WORKSPACE/tools/gluten-it - $MVN_CMD clean install -P${{ matrix.spark }} - - name: Build and run TPC-H / TPC-DS - run: | - cd $GITHUB_WORKSPACE/tools/gluten-it - GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.IOThreads=16 - GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ - --extra-conf=spark.gluten.sql.columnar.backend.velox.IOThreads=16 + path: .m2/repository/org/apache/arrow/ + - name: Build and run TPCH/DS tests + run: | + docker pull centos:7 + docker run -v $GITHUB_WORKSPACE:/work -v /$GITHUB_WORKSPACE/.m2:/root/.m2/ -w /work \ + -e matrix.java=${{ matrix.java }} -e matrix.spark=${{ matrix.spark }} \ + centos:7 \ + bash -c " + sed -i -e 's|mirrorlist=|#mirrorlist=|g' /etc/yum.repos.d/CentOS-* || true + sed -i -e 's|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g' /etc/yum.repos.d/CentOS-* || true + + # Setup java and maven + yum update -y && yum install -y java-1.8.0-openjdk-devel wget + + wget -nv https://downloads.apache.org/maven/maven-3/3.8.8/binaries/apache-maven-3.8.8-bin.tar.gz + tar -xvf apache-maven-3.8.8-bin.tar.gz && mv apache-maven-3.8.8 /usr/lib/maven + export PATH=${PATH}:/usr/lib/maven/bin + + # Set environment variables + export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk + + # Build gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests + cd /work/tools/gluten-it + mvn -ntp clean install -P${{ matrix.spark }} -P${{ matrix.java }} + + # Run TPC-H / TPC-DS + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 + + # Run TPC-H / TPC-DS with RAS + cd /work/tools/gluten-it + GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true \ + && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + --extra-conf=spark.gluten.ras.enabled=true + " run-tpc-test-ubuntu-oom: needs: build-native-lib-centos-7 @@ -464,6 +491,8 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] + uniffle: [ "0.9.1" ] + hadoop: [ "2.8.5" ] runs-on: ubuntu-20.04 container: centos:8 steps: @@ -486,21 +515,16 @@ jobs: run: | yum update -y && yum install -y java-1.8.0-openjdk-devel wget git $SETUP install_maven - - name: Build for Uniffle 0.9.0 + - name: Install Uniffle ${{ matrix.uniffle }} run: | export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd /opt && \ - git clone -b v0.9.0 https://github.com/apache/incubator-uniffle.git && \ - cd incubator-uniffle && \ - $MVN_CMD clean install -Phadoop2.8,spark3 -DskipTests - cd /opt && \ - ${WGET_CMD} https://archive.apache.org/dist/incubator/uniffle/0.9.0/apache-uniffle-0.9.0-incubating-bin.tar.gz && \ - tar xzf apache-uniffle-0.9.0-incubating-bin.tar.gz -C /opt/ && mv /opt/rss-0.9.0-hadoop2.8 /opt/uniffle && \ - ${WGET_CMD} https://archive.apache.org/dist/hadoop/common/hadoop-2.8.5/hadoop-2.8.5.tar.gz && \ - tar xzf hadoop-2.8.5.tar.gz -C /opt/ - rm -rf /opt/incubator-uniffle + ${WGET_CMD} https://archive.apache.org/dist/incubator/uniffle/${{ matrix.uniffle }}/apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz && \ + mkdir /opt/uniffle && tar xzf apache-uniffle-${{ matrix.uniffle }}-incubating-bin.tar.gz -C /opt/uniffle --strip-components=1 && \ + ${WGET_CMD} https://archive.apache.org/dist/hadoop/common/hadoop-${{ matrix.hadoop }}/hadoop-${{ matrix.hadoop }}.tar.gz && \ + tar xzf hadoop-${{ matrix.hadoop }}.tar.gz -C /opt/ && \ cd /opt/uniffle && mkdir shuffle_data && \ - bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-2.8.5' > ./bin/rss-env.sh" && \ + bash -c "echo -e 'XMX_SIZE=16g\nHADOOP_HOME=/opt/hadoop-${{ matrix.hadoop }}' > ./bin/rss-env.sh" && \ bash -c "echo -e 'rss.coordinator.shuffle.nodes.max 1\nrss.rpc.server.port 19999' > ./conf/coordinator.conf" && \ bash -c "echo -e 'rss.server.app.expired.withoutHeartbeat 7200000\nrss.server.heartbeat.delay 3000\nrss.rpc.server.port 19997\nrss.rpc.server.type GRPC_NETTY\nrss.jetty.http.port 19996\nrss.server.netty.port 19995\nrss.storage.basePath /opt/uniffle/shuffle_data\nrss.storage.type MEMORY_LOCALFILE\nrss.coordinator.quorum localhost:19999\nrss.server.flush.thread.alive 10\nrss.server.single.buffer.flush.threshold 64m' > ./conf/server.conf" && \ bash ./bin/start-coordinator.sh && bash ./bin/start-shuffle-server.sh @@ -508,11 +532,11 @@ jobs: run: | cd $GITHUB_WORKSPACE/ && \ $MVN_CMD clean install -P${{ matrix.spark }} -Pbackends-velox -Puniffle -DskipTests - - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle-0.9.0 + - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.2 with uniffle-${{ matrix.uniffle }} run: | export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk && \ cd $GITHUB_WORKSPACE/tools/gluten-it && \ - $MVN_CMD clean install -Pspark-3.2 -Puniffle && \ + $MVN_CMD clean install -P${{ matrix.spark }} -Puniffle && \ GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ --local --preset=velox-with-uniffle --benchmark-type=h --error-on-memleak --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 @@ -522,7 +546,7 @@ jobs: fail-fast: false matrix: spark: [ "spark-3.2" ] - celeborn: [ "celeborn-0.5.1", "celeborn-0.4.2", "celeborn-0.3.2-incubating" ] + celeborn: [ "celeborn-0.5.2", "celeborn-0.4.2", "celeborn-0.3.2-incubating" ] runs-on: ubuntu-20.04 container: ubuntu:22.04 steps: @@ -555,7 +579,7 @@ jobs: EXTRA_PROFILE="" if [ "${{ matrix.celeborn }}" = "celeborn-0.4.2" ]; then EXTRA_PROFILE="-Pceleborn-0.4" - elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.1" ]; then + elif [ "${{ matrix.celeborn }}" = "celeborn-0.5.2" ]; then EXTRA_PROFILE="-Pceleborn-0.5" fi echo "EXTRA_PROFILE: ${EXTRA_PROFILE}" @@ -586,14 +610,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.2.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.2 @@ -615,6 +631,12 @@ jobs: with: name: test-report-spark32 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark32 + path: /tmp/tpch-approved-plan/** run-spark-test-spark32-slow: needs: build-native-lib-centos-7 @@ -632,14 +654,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.2.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.2 @@ -671,14 +685,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.3.1 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.3 @@ -700,6 +706,12 @@ jobs: with: name: test-report-spark33 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark33 + path: /tmp/tpch-approved-plan/** run-spark-test-spark33-slow: @@ -718,14 +730,6 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Prepare spark.test.home for Spark 3.3.1 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.3 @@ -758,15 +762,7 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.4.4 (other tests) + - name: Prepare spark.test.home for Spark 3.4.3 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.4 dnf module -y install python39 && \ @@ -787,6 +783,12 @@ jobs: with: name: test-report-spark34 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark34 + path: /tmp/tpch-approved-plan/** run-spark-test-spark34-slow: @@ -805,15 +807,7 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.4.4 (slow tests) + - name: Prepare spark.test.home for Spark 3.4.3 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.4 - name: Build and Run unit test for Spark 3.4.4 (slow tests) @@ -845,27 +839,19 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 (other tests) + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags - name: Upload test report @@ -874,6 +860,12 @@ jobs: with: name: test-report-spark35 path: '**/surefire-reports/TEST-*.xml' + - name: Upload golden files + if: failure() + uses: actions/upload-artifact@v4 + with: + name: golden-files-spark35 + path: /tmp/tpch-approved-plan/** run-spark-test-spark35-scala213: needs: build-native-lib-centos-7 @@ -891,23 +883,15 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5-scala2.13 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: Build and Run unit test for Spark 3.5.3 with scala-2.13 (other tests) + - name: Build and Run unit test for Spark 3.5.2 with scala-2.13 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.13 @@ -937,21 +921,13 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: Build and Run unit test for Spark 3.5.3 (slow tests) + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ - $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ + $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Phudi -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - name: Upload test report @@ -977,32 +953,22 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.5.3 (other tests) + - name: Prepare spark.test.home for Spark 3.5.2 (other tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 dnf module -y install python39 && \ alternatives --set python3 /usr/bin/python3.9 && \ pip3 install setuptools && \ - pip3 install pyspark==3.5.3 cython && \ + pip3 install pyspark==3.5.2 cython && \ pip3 install pandas pyarrow - - name: (To be fixed) Build and Run unit test for Spark 3.5.3 (other tests) - continue-on-error: true + - name: Build and Run unit test for Spark 3.5.2 (other tests) run: | cd $GITHUB_WORKSPACE/ export SPARK_SCALA_VERSION=2.12 $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/ -Dspark.gluten.ras.enabled=true" \ -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.SkipTestTags - - name: (To be enabled) Upload test report - if: false + - name: Upload test report uses: actions/upload-artifact@v4 with: name: test-report-spark35-ras @@ -1024,26 +990,16 @@ jobs: with: name: arrow-jars-centos-7-${{github.sha}} path: /root/.m2/repository/org/apache/arrow/ - - name: Update mirror list - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - - name: Setup build dependency - run: | - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - - name: Prepare spark.test.home for Spark 3.5.3 (slow tests) + - name: Prepare spark.test.home for Spark 3.5.2 (slow tests) run: | bash .github/workflows/util/install_spark_resources.sh 3.5 - - name: (To be fixed) Build and Run unit test for Spark 3.5.3 (slow tests) - continue-on-error: true + - name: Build and Run unit test for Spark 3.5.2 (slow tests) run: | cd $GITHUB_WORKSPACE/ $MVN_CMD clean test -Pspark-3.5 -Pbackends-velox -Pceleborn -Piceberg -Pdelta -Pspark-ut \ -DargLine="-Dspark.test.home=$GITHUB_WORKSPACE//shims/spark35/spark_home/ -Dspark.gluten.ras.enabled=true" \ -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest - - name: (To be enabled) Upload test report - if: false + - name: Upload test report uses: actions/upload-artifact@v4 with: name: test-report-spark35-slow-ras @@ -1061,22 +1017,17 @@ jobs: key: ccache-centos8-release-default-${{github.sha}} restore-keys: | ccache-centos8-release-default - - name: Setup java and maven - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - yum install sudo patch java-1.8.0-openjdk-devel wget -y - $SETUP install_maven - name: Build Gluten native libraries run: | df -a bash dev/ci-velox-buildshared-centos-8.sh - - name: "Save ccache" - uses: actions/cache/save@v3 - id: ccache - with: - path: '${{ env.CCACHE_DIR }}' - key: ccache-centos8-release-default-${{github.sha}} + ccache -s + # - name: "Save ccache" + # uses: actions/cache/save@v3 + # id: ccache + # with: + # path: '${{ env.CCACHE_DIR }}' + # key: ccache-centos8-release-default-${{github.sha}} - name: Run CPP unit test run: | cd ./cpp/build && ctest -V @@ -1084,9 +1035,11 @@ jobs: run: | $MVN_CMD test -Pspark-3.5 -Pbackends-velox -pl backends-velox -am \ -DtagsToInclude="org.apache.gluten.tags.GenerateExample" -Dtest=none -DfailIfNoTests=false -Dexec.skip - # This test depends on example.json generated by the above mvn test. - cd cpp/build/velox/benchmarks && sudo chmod +x ./generic_benchmark - ./generic_benchmark --run-example --with-shuffle --threads 1 --iterations 1 + # This test depends on files generated by the above mvn test. + ./cpp/build/velox/benchmarks/generic_benchmark --with-shuffle --partitioning hash --threads 1 --iterations 1 \ + --conf $(realpath backends-velox/generated-native-benchmark/conf_12_0.ini) \ + --plan $(realpath backends-velox/generated-native-benchmark/plan_12_0.json) \ + --data $(realpath backends-velox/generated-native-benchmark/data_12_0_0.parquet),$(realpath backends-velox/generated-native-benchmark/data_12_0_1.parquet) - name: Run UDF test run: | # Depends on --build_example=ON. diff --git a/.github/workflows/velox_backend_cache.yml b/.github/workflows/velox_backend_cache.yml index 71a7a5840b4e..3a6a8fadca0e 100644 --- a/.github/workflows/velox_backend_cache.yml +++ b/.github/workflows/velox_backend_cache.yml @@ -31,9 +31,8 @@ concurrency: jobs: cache-native-lib-centos-7: runs-on: ubuntu-20.04 - container: apache/gluten:vcpkg-centos-7 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Get Ccache uses: actions/cache/restore@v3 with: @@ -43,9 +42,11 @@ jobs: ccache-centos7-release-default - name: Build Gluten native libraries run: | - df -a - yum install ccache -y - bash dev/ci-velox-buildstatic-centos-7.sh + docker run -v $GITHUB_WORKSPACE:/work -w /work apache/gluten:vcpkg-centos-7 bash -c " + export CCACHE_DIR=/work/.ccache + mkdir -p /work/.ccache + bash dev/ci-velox-buildstatic-centos-7.sh + " - name: Save Ccache uses: actions/cache/save@v3 id: ccache @@ -65,12 +66,6 @@ jobs: key: ccache-centos8-release-default-${{github.sha}} restore-keys: | ccache-centos8-release-default - - name: Setup java and maven - run: | - sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true - sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true - yum install sudo patch java-1.8.0-openjdk-devel wget -y - bash .github/workflows/util/setup_helper.sh install_maven - name: Build Gluten native libraries run: | df -a diff --git a/.github/workflows/velox_weekly.yml b/.github/workflows/velox_weekly.yml index 8cd74723f0ae..fa8273d51a2f 100644 --- a/.github/workflows/velox_weekly.yml +++ b/.github/workflows/velox_weekly.yml @@ -38,7 +38,6 @@ jobs: runs-on: ubuntu-20.04 container: ${{ matrix.os }} steps: - - uses: actions/checkout@v2 - name: Update mirror list run: | sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true @@ -72,7 +71,7 @@ jobs: mv apache-maven-3.8.8 /usr/lib/maven && \ export MAVEN_HOME=/usr/lib/maven && \ export PATH=${PATH}:${MAVEN_HOME}/bin && \ - cd $GITHUB_WORKSPACE/ && \ + git clone -b main --depth=1 https://github.com/apache/incubator-gluten.git && cd incubator-gluten/ ./dev/package.sh build-on-ubuntu: @@ -93,8 +92,8 @@ jobs: sudo apt install -y software-properties-common sudo add-apt-repository ppa:ubuntu-toolchain-r/test sudo apt update && sudo apt install -y gcc-11 g++-11 - sudo ln -sf /usr/bin/gcc-11 /usr/bin/gcc - sudo ln -sf /usr/bin/g++-11 /usr/bin/g++ + export CC=/usr/bin/gcc-11 + export CXX=/usr/bin/g++-11 fi sudo apt-get install -y openjdk-8-jdk export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-amd64 diff --git a/.gitignore b/.gitignore index d30ea813ac04..4ea83cbf7e12 100644 --- a/.gitignore +++ b/.gitignore @@ -95,3 +95,6 @@ dist/ /cpp-ch/local-engine/Parser/*_udf !/cpp-ch/local-engine/Parser/example_udf + +# For Hive +metastore_db/ diff --git a/.idea/icon.svg b/.idea/icon.svg new file mode 100644 index 000000000000..3a8604285c28 --- /dev/null +++ b/.idea/icon.svg @@ -0,0 +1,7 @@ + + + + Layer 1 + + + \ No newline at end of file diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index b00470f9ea4a..3a4a9422479b 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -13,6 +13,70 @@ jar Gluten Backends ClickHouse + + + iceberg + + false + + + + org.apache.gluten + gluten-iceberg + ${project.version} + + + org.apache.iceberg + iceberg-spark-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + provided + + + org.apache.iceberg + iceberg-spark-runtime-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-iceberg-sources + generate-sources + + add-source + + + + ${project.basedir}/src/main-iceberg/scala + ${project.basedir}/src/main-iceberg/java + + + + + add-iceberg-test-sources + generate-test-sources + + add-test-source + + + + ${project.basedir}/src/test-iceberg/scala + ${project.basedir}/src/test-iceberg/java + + + + + + + + + + org.apache.gluten @@ -368,6 +432,7 @@ src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/*.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/merge/*.scala + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/stats/*.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/DeltaLog.scala src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/Snapshot.scala @@ -425,6 +490,18 @@ + + add-test-sources + generate-test-sources + + add-test-source + + + + src/test/delta-${delta.binary.version} + + + diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 1ea6fe249e81..06b6fa8c4c20 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val snapshot: Snapshot)(implicit override val clock: Clock) diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..a360fa8d7291 --- /dev/null +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.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.delta.rules + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.metering.DeltaLogging + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging { + + // For Delta 2.0, it can not support to optimize query with the metadata + override def apply(plan: LogicalPlan): LogicalPlan = plan +} diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 2a97ef951506..11acfb026054 100644 --- a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -36,7 +36,6 @@ import org.apache.commons.lang3.exception.ExceptionUtils import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val snapshot: Snapshot)(implicit override val clock: Clock) diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..dbb5c4050a2c --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala @@ -0,0 +1,77 @@ +/* + * 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.delta.rules + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper} +import org.apache.spark.sql.delta.files.TahoeLogFileIndex +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaScanGenerator + +import org.apache.hadoop.fs.Path + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging + with SubqueryTransformerHelper + with OptimizeMetadataOnlyDeltaQuery { + + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + OptimisticTransaction + .getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max + if ( + spark.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE, + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE) + .toBoolean && + !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true) + ) { + optimizeQueryWithMetadata(plan) + } else { + plan + } + } +} diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala new file mode 100644 index 000000000000..21e31d35411e --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/stats/PrepareDeltaScan.scala @@ -0,0 +1,406 @@ +/* + * 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.delta.stats + +import java.util.Objects + +import scala.collection.mutable + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.files.{TahoeFileIndexWithSnapshot, TahoeLogFileIndex} +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.PROJECT +import org.apache.spark.sql.execution.datasources.LogicalRelation + +/** + * Gluten overwrite Delta: + * + * This file is copied from Delta 2.3.0, it is modified to overcome the following issues: + * 1. Returns the plan directly even if stats.skipping is turned off + */ + +/** + * Before query planning, we prepare any scans over delta tables by pushing + * any projections or filters in allowing us to gather more accurate statistics + * for CBO and metering. + * + * Note the following + * - This rule also ensures that all reads from the same delta log use the same snapshot of log + * thus providing snapshot isolation. + * - If this rule is invoked within an active [[OptimisticTransaction]], then the scans are + * generated using the transaction. + */ +trait PrepareDeltaScanBase extends Rule[LogicalPlan] + with PredicateHelper + with DeltaLogging + with OptimizeMetadataOnlyDeltaQuery + with PreprocessTableWithDVs { self: PrepareDeltaScan => + + /** + * Tracks the first-access snapshots of other logs planned by this rule. The snapshots are + * the keyed by the log's unique id. Note that the lifetime of this rule is a single + * query, therefore, the map tracks the snapshots only within a query. + */ + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + /** + * Gets the [[DeltaScanGenerator]] for the given log, which will be used to generate + * [[DeltaScan]]s. Every time this method is called on a log within the lifetime of this + * rule (i.e., the lifetime of the query for which this rule was instantiated), the returned + * generator will read a snapshot that is pinned on the first access for that log. + * + * Internally, it will use the snapshot of the file index, the snapshot of the active transaction + * (if any), or the latest snapshot of the given log. + */ + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + val scanGenerator = OptimisticTransaction.getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + import PrepareDeltaScanBase._ + if (onGetDeltaScanGeneratorCallback != null) onGetDeltaScanGeneratorCallback(scanGenerator) + scanGenerator + } + + /** + * Helper method to generate a [[PreparedDeltaFileIndex]] + */ + protected def getPreparedIndex( + preparedScan: DeltaScan, + fileIndex: TahoeLogFileIndex): PreparedDeltaFileIndex = { + assert(fileIndex.partitionFilters.isEmpty, + "Partition filters should have been extracted by DeltaAnalysis.") + PreparedDeltaFileIndex( + spark, + fileIndex.deltaLog, + fileIndex.path, + preparedScan, + fileIndex.versionToUse) + } + + /** + * Scan files using the given `filters` and return `DeltaScan`. + * + * Note: when `limitOpt` is non empty, `filters` must contain only partition filters. Otherwise, + * it can contain arbitrary filters. See `DeltaTableScan` for more details. + */ + protected def filesForScan( + scanGenerator: DeltaScanGenerator, + limitOpt: Option[Int], + filters: Seq[Expression], + delta: LogicalRelation): DeltaScan = { + withStatusCode("DELTA", "Filtering files for query") { + if (limitOpt.nonEmpty) { + // If we trigger limit push down, the filters must be partition filters. Since + // there are no data filters, we don't need to apply Generated Columns + // optimization. See `DeltaTableScan` for more details. + return scanGenerator.filesForScan(limitOpt.get, filters) + } + val filtersForScan = + if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) { + filters + } else { + val generatedPartitionFilters = GeneratedColumn.generatePartitionFilters( + spark, scanGenerator.snapshotToScan, filters, delta) + filters ++ generatedPartitionFilters + } + scanGenerator.filesForScan(filtersForScan) + } + } + + /** + * Prepares delta scans sequentially. + */ + protected def prepareDeltaScan(plan: LogicalPlan): LogicalPlan = { + // A map from the canonicalized form of a DeltaTableScan operator to its corresponding delta + // scan. This map is used to avoid fetching duplicate delta indexes for structurally-equal + // delta scans. + val deltaScans = new mutable.HashMap[LogicalPlan, DeltaScan]() + + transformWithSubqueries(plan) { + case scan @ DeltaTableScan(planWithRemovedProjections, filters, fileIndex, + limit, delta) => + val scanGenerator = getDeltaScanGenerator(fileIndex) + val preparedScan = deltaScans.getOrElseUpdate(planWithRemovedProjections.canonicalized, + filesForScan(scanGenerator, limit, filters, delta)) + val preparedIndex = getPreparedIndex(preparedScan, fileIndex) + optimizeGeneratedColumns(scan, preparedIndex, filters, limit, delta) + } + } + + protected def optimizeGeneratedColumns( + scan: LogicalPlan, + preparedIndex: PreparedDeltaFileIndex, + filters: Seq[Expression], + limit: Option[Int], + delta: LogicalRelation): LogicalPlan = { + if (limit.nonEmpty) { + // If we trigger limit push down, the filters must be partition filters. Since + // there are no data filters, we don't need to apply Generated Columns + // optimization. See `DeltaTableScan` for more details. + return DeltaTableUtils.replaceFileIndex(scan, preparedIndex) + } + if (!GeneratedColumn.partitionFilterOptimizationEnabled(spark)) { + DeltaTableUtils.replaceFileIndex(scan, preparedIndex) + } else { + val generatedPartitionFilters = + GeneratedColumn.generatePartitionFilters(spark, preparedIndex, filters, delta) + val scanWithFilters = + if (generatedPartitionFilters.nonEmpty) { + scan transformUp { + case delta @ DeltaTable(_: TahoeLogFileIndex) => + Filter(generatedPartitionFilters.reduceLeft(And), delta) + } + } else { + scan + } + DeltaTableUtils.replaceFileIndex(scanWithFilters, preparedIndex) + } + } + + override def apply(_plan: LogicalPlan): LogicalPlan = { + var plan = _plan + + // --- modified start + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + + val shouldPrepareDeltaScan = ( + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING) + ) + val updatedPlan = if (shouldPrepareDeltaScan) { + if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_METADATA_QUERY_ENABLED)) { + plan = optimizeQueryWithMetadata(plan) + } + prepareDeltaScan(plan) + } else { + // If this query is running inside an active transaction and is touching the same table + // as the transaction, then mark that the entire table as tainted to be safe. + OptimisticTransaction.getActive.foreach { txn => + val logsInPlan = plan.collect { case DeltaTable(fileIndex) => fileIndex.deltaLog } + if (logsInPlan.exists(_.isSameLogAs(txn.deltaLog))) { + txn.readWholeTable() + } + } + + // Just return the plan if statistics based skipping is off. + // It will fall back to just partition pruning at planning time. + plan + } + // --- modified end + preprocessTablesWithDVs(updatedPlan) + } + + /** + * This is an extractor object. See https://docs.scala-lang.org/tour/extractor-objects.html. + */ + object DeltaTableScan { + + /** + * The components of DeltaTableScanType are: + * - the plan with removed projections. We remove projections as a plan differentiator + * because it does not affect file listing results. + * - filter expressions collected by `PhysicalOperation` + * - the `TahoeLogFileIndex` of the matched DeltaTable` + * - integer value of limit expression, if any + * - matched `DeltaTable` + */ + private type DeltaTableScanType = + (LogicalPlan, Seq[Expression], TahoeLogFileIndex, Option[Int], LogicalRelation) + + /** + * This is an extractor method (basically, the opposite of a constructor) which takes in an + * object `plan` and tries to give back the arguments as a [[DeltaTableScanType]]. + */ + def unapply(plan: LogicalPlan): Option[DeltaTableScanType] = { + val limitPushdownEnabled = spark.conf.get(DeltaSQLConf.DELTA_LIMIT_PUSHDOWN_ENABLED) + + // Remove projections as a plan differentiator because it does not affect file listing + // results. Plans with the same filters but different projections therefore will not have + // duplicate delta indexes. + def canonicalizePlanForDeltaFileListing(plan: LogicalPlan): LogicalPlan = { + val planWithRemovedProjections = plan.transformWithPruning(_.containsPattern(PROJECT)) { + case p: Project if p.projectList.forall(_.isInstanceOf[AttributeReference]) => p.child + } + planWithRemovedProjections + } + + plan match { + case LocalLimit(IntegerLiteral(limit), + PhysicalOperation(_, filters, delta @ DeltaTable(fileIndex: TahoeLogFileIndex))) + if limitPushdownEnabled && containsPartitionFiltersOnly(filters, fileIndex) => + Some((canonicalizePlanForDeltaFileListing(plan), filters, fileIndex, Some(limit), delta)) + case PhysicalOperation( + _, + filters, + delta @ DeltaTable(fileIndex: TahoeLogFileIndex)) => + val allFilters = fileIndex.partitionFilters ++ filters + Some((canonicalizePlanForDeltaFileListing(plan), allFilters, fileIndex, None, delta)) + + case _ => None + } + } + + private def containsPartitionFiltersOnly( + filters: Seq[Expression], + fileIndex: TahoeLogFileIndex): Boolean = { + val partitionColumns = fileIndex.snapshotAtAnalysis.metadata.partitionColumns + import DeltaTableUtils._ + filters.forall(expr => !containsSubquery(expr) && + isPredicatePartitionColumnsOnly(expr, partitionColumns, spark)) + } + } +} + +class PrepareDeltaScan(protected val spark: SparkSession) + extends PrepareDeltaScanBase + +object PrepareDeltaScanBase { + + /** + * Optional callback function that is called after `getDeltaScanGenerator` is called + * by the PrepareDeltaScan rule. This is primarily used for testing purposes. + */ + @volatile private var onGetDeltaScanGeneratorCallback: DeltaScanGenerator => Unit = _ + + /** + * Run a thunk of code with the given callback function injected into the PrepareDeltaScan rule. + * The callback function is called after `getDeltaScanGenerator` is called + * by the PrepareDeltaScan rule. This is primarily used for testing purposes. + */ + private[delta] def withCallbackOnGetDeltaScanGenerator[T]( + callback: DeltaScanGenerator => Unit)(thunk: => T): T = { + try { + onGetDeltaScanGeneratorCallback = callback + thunk + } finally { + onGetDeltaScanGeneratorCallback = null + } + } +} + +/** + * A [[TahoeFileIndex]] that uses a prepared scan to return the list of relevant files. + * This is injected into a query right before query planning by [[PrepareDeltaScan]] so that + * CBO and metering can accurately understand how much data will be read. + * + * @param versionScanned The version of the table that is being scanned, if a specific version + * has specifically been requested, e.g. by time travel. + */ +case class PreparedDeltaFileIndex( + override val spark: SparkSession, + override val deltaLog: DeltaLog, + override val path: Path, + preparedScan: DeltaScan, + versionScanned: Option[Long]) + extends TahoeFileIndexWithSnapshot(spark, deltaLog, path, preparedScan.scannedSnapshot) + with DeltaLogging { + + /** + * Returns all matching/valid files by the given `partitionFilters` and `dataFilters` + */ + override def matchingFiles( + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): Seq[AddFile] = { + val currentFilters = ExpressionSet(partitionFilters ++ dataFilters) + val (addFiles, eventData) = if (currentFilters == preparedScan.allFilters || + currentFilters == preparedScan.filtersUsedForSkipping) { + // [[DeltaScan]] was created using `allFilters` out of which only `filtersUsedForSkipping` + // filters were used for skipping while creating the DeltaScan. + // If currentFilters is same as allFilters, then no need to recalculate files and we can use + // previous results. + // If currentFilters is same as filtersUsedForSkipping, then also we don't need to recalculate + // files as [[DeltaScan.files]] were calculates using filtersUsedForSkipping only. So if we + // recalculate, we will get same result. So we should use previous result in this case also. + val eventData = Map( + "reused" -> true, + "currentFiltersSameAsPreparedAllFilters" -> (currentFilters == preparedScan.allFilters), + "currentFiltersSameAsPreparedFiltersUsedForSkipping" -> + (currentFilters == preparedScan.filtersUsedForSkipping) + ) + (preparedScan.files.distinct, eventData) + } else { + logInfo( + s""" + |Prepared scan does not match actual filters. Reselecting files to query. + |Prepared: ${preparedScan.allFilters} + |Actual: ${currentFilters} + """.stripMargin) + val eventData = Map( + "reused" -> false, + "preparedAllFilters" -> preparedScan.allFilters.mkString(","), + "preparedFiltersUsedForSkipping" -> preparedScan.filtersUsedForSkipping.mkString(","), + "currentFilters" -> currentFilters.mkString(",") + ) + val files = preparedScan.scannedSnapshot.filesForScan(partitionFilters ++ dataFilters).files + (files, eventData) + } + recordDeltaEvent(deltaLog, + opType = "delta.preparedDeltaFileIndex.reuseSkippingResult", + data = eventData) + addFiles + } + + /** + * Returns the list of files that will be read when scanning this relation. This call may be + * very expensive for large tables. + */ + override def inputFiles: Array[String] = + preparedScan.files.map(f => absolutePath(f.path).toString).toArray + + /** Refresh any cached file listings */ + override def refresh(): Unit = { } + + /** Sum of table file sizes, in bytes */ + override def sizeInBytes: Long = + preparedScan.scanned.bytesCompressed + .getOrElse(spark.sessionState.conf.defaultSizeInBytes) + + override def equals(other: Any): Boolean = other match { + case p: PreparedDeltaFileIndex => + p.deltaLog == deltaLog && p.path == path && p.preparedScan == preparedScan && + p.partitionSchema == partitionSchema && p.versionScanned == versionScanned + case _ => false + } + + override def hashCode(): Int = { + Objects.hash(deltaLog, path, preparedScan, partitionSchema, versionScanned) + } + +} diff --git a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala index e747c87c6a67..ba4c21df3a34 100644 --- a/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala +++ b/backends-clickhouse/src/main/delta-32/io/delta/tables/ClickhouseTable.scala @@ -27,12 +27,7 @@ import scala.collection.JavaConverters._ class ClickhouseTable( @transient private val _df: Dataset[Row], @transient private val table: ClickHouseTableV2) - extends DeltaTable(_df, table) { - - override def optimize(): DeltaOptimizeBuilder = { - DeltaOptimizeBuilder(table) - } -} + extends DeltaTable(_df, table) {} object ClickhouseTable { diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index 2f5824b58092..05f7fdbfa423 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.delta +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.spark.SparkException @@ -24,12 +25,11 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.constraints.{Constraint, Constraints} -import org.apache.spark.sql.delta.files.{DelayedCommitProtocol, DeltaFileFormatWriter, MergeTreeDelayedCommitProtocol, TransactionalWrite} +import org.apache.spark.sql.delta.files._ import org.apache.spark.sql.delta.hooks.AutoCompact import org.apache.spark.sql.delta.schema.{InnerInvariantViolationException, InvariantViolationException} import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker -import org.apache.spark.sql.execution.{CHDelayedCommitProtocol, QueryExecution, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormatWriter, GlutenWriterColumnarRules, WriteFiles, WriteJobStatsTracker} import org.apache.spark.sql.execution.datasources.v1.MergeTreeWriterInjects @@ -43,14 +43,15 @@ import org.apache.hadoop.fs.Path import scala.collection.mutable.ListBuffer -object ClickhouseOptimisticTransaction {} - class ClickhouseOptimisticTransaction( override val deltaLog: DeltaLog, override val catalogTable: Option[CatalogTable], override val snapshot: Snapshot) extends OptimisticTransaction(deltaLog, catalogTable, snapshot) { + private lazy val writingMergeTree = + ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration) + def this( deltaLog: DeltaLog, catalogTable: Option[CatalogTable], @@ -65,120 +66,137 @@ class ClickhouseOptimisticTransaction( override def writeFiles( inputData: Dataset[_], writeOptions: Option[DeltaOptions], + isOptimize: Boolean, additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { - hasWritten = true - - val spark = inputData.sparkSession - val (data, partitionSchema) = performCDCPartition(inputData) - val outputPath = deltaLog.dataPath - - val (queryExecution, output, generatedColumnConstraints, _) = - normalizeData(deltaLog, writeOptions, data) - - val tableV2 = ClickHouseTableV2.getTable(deltaLog) - val committer = - new MergeTreeDelayedCommitProtocol( - outputPath.toString, - None, - None, - tableV2.dataBaseName, - tableV2.tableName) - - // val (optionalStatsTracker, _) = - // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) - val (optionalStatsTracker, _) = (None, None) - - val constraints = - Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints - - SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { - val queryPlan = queryExecution.executedPlan - val (newQueryPlan, newOutput) = - MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output) - val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput) - val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput) - - val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() - - if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { - val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( - new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), - BasicWriteJobStatsTracker.metrics) - // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) - statsTrackers.append(basicWriteJobStatsTracker) - } - - // Iceberg spec requires partition columns in data files - val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata) - // Retain only a minimal selection of Spark writer options to avoid any potential - // compatibility issues - var options = (writeOptions match { - case None => Map.empty[String, String] - case Some(writeOptions) => - writeOptions.options.filterKeys { - key => - key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || - key.equalsIgnoreCase(DeltaOptions.COMPRESSION) - }.toMap - }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString) - - spark.conf.getAll.foreach( - entry => { - if ( - CHConf.startWithSettingsPrefix(entry._1) - || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) - ) { - options += (entry._1 -> entry._2) - } - }) - - try { - val format = tableV2.getFileFormat(protocol, metadata) - GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName())) - MergeTreeFileFormatWriter.write( - sparkSession = spark, - plan = newQueryPlan, - fileFormat = format, - // formats. - committer = committer, - outputSpec = outputSpec, - // scalastyle:off deltahadoopconfiguration - hadoopConf = spark.sessionState - .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), - // scalastyle:on deltahadoopconfiguration - partitionColumns = partitioningColumns, - bucketSpec = - tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver), - statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, - options = options, - constraints = constraints - ) - } catch { - case s: SparkException => - // Pull an InvariantViolationException up to the top level if it was the root cause. - val violationException = ExceptionUtils.getRootCause(s) - if (violationException.isInstanceOf[InvariantViolationException]) { - throw violationException - } else { - throw s - } - } finally { - GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None) - } + val nativeWrite = GlutenConfig.getConf.enableNativeWriter.getOrElse(false) + if (writingMergeTree) { + if (isOptimize) { + throw new UnsupportedOperationException("Optimize is not supported for ClickHouse") } - committer.addedStatuses.toSeq ++ committer.changeFiles + // TODO: update FallbackByBackendSettings for mergetree always return true + val onePipeline = nativeWrite && CHConf.get.enableOnePipelineMergeTreeWrite + if (onePipeline) + pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints) + else + writeMergeTree(inputData, writeOptions, additionalConstraints) } else { - // TODO: support native delta parquet write - // 1. insert FakeRowAdaptor - // 2. DeltaInvariantCheckerExec transform - // 3. DeltaTaskStatisticsTracker collect null count / min values / max values - // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeApplicable', - // 'nativeFormat' in the LocalProperty of the sparkcontext - super.writeFiles(inputData, writeOptions, additionalConstraints) + if (isOptimize || !nativeWrite) { + super.writeFiles(inputData, writeOptions, isOptimize, additionalConstraints) + } else { + pipelineWriteFiles(inputData, writeOptions, isOptimize = false, additionalConstraints) + } } } + @deprecated("Use pipelineWriteFiles instead") + private def writeMergeTree( + inputData: Dataset[_], + writeOptions: Option[DeltaOptions], + additionalConstraints: Seq[Constraint]): Seq[FileAction] = { + + hasWritten = true + + val spark = inputData.sparkSession + val (data, partitionSchema) = performCDCPartition(inputData) + val outputPath = deltaLog.dataPath + + val (queryExecution, output, generatedColumnConstraints, _) = + normalizeData(deltaLog, writeOptions, data) + + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + val committer = + new MergeTreeDelayedCommitProtocol( + outputPath.toString, + None, + None, + tableV2.dataBaseName, + tableV2.tableName) + + // val (optionalStatsTracker, _) = + // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) + val (optionalStatsTracker, _) = (None, None) + + val constraints = + Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints + + SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { + val queryPlan = queryExecution.executedPlan + val (newQueryPlan, newOutput) = + MergeTreeWriterInjects.insertFakeRowAdaptor(queryPlan, output) + val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, newOutput) + val partitioningColumns = getPartitioningColumns(partitionSchema, newOutput) + + val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( + new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), + BasicWriteJobStatsTracker.metrics) + // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + statsTrackers.append(basicWriteJobStatsTracker) + } + + // Iceberg spec requires partition columns in data files + val writePartitionColumns = IcebergCompat.isAnyEnabled(metadata) + // Retain only a minimal selection of Spark writer options to avoid any potential + // compatibility issues + var options = (writeOptions match { + case None => Map.empty[String, String] + case Some(writeOptions) => + writeOptions.options.filterKeys { + key => + key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || + key.equalsIgnoreCase(DeltaOptions.COMPRESSION) + }.toMap + }) + (DeltaOptions.WRITE_PARTITION_COLUMNS -> writePartitionColumns.toString) + + spark.conf.getAll.foreach( + entry => { + if ( + CHConf.startWithSettingsPrefix(entry._1) + || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) + ) { + options += (entry._1 -> entry._2) + } + }) + + try { + val format = tableV2.getFileFormat(protocol, metadata) + GlutenWriterColumnarRules.injectSparkLocalProperty(spark, Some(format.shortName())) + MergeTreeFileFormatWriter.write( + sparkSession = spark, + plan = newQueryPlan, + fileFormat = format, + // formats. + committer = committer, + outputSpec = outputSpec, + // scalastyle:off deltahadoopconfiguration + hadoopConf = spark.sessionState + .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), + // scalastyle:on deltahadoopconfiguration + partitionColumns = partitioningColumns, + bucketSpec = + tableV2.normalizedBucketSpec(output.map(_.name), spark.sessionState.conf.resolver), + statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, + options = options, + constraints = constraints + ) + } catch { + case s: SparkException => + // Pull an InvariantViolationException up to the top level if it was the root cause. + val violationException = ExceptionUtils.getRootCause(s) + if (violationException.isInstanceOf[InvariantViolationException]) { + throw violationException + } else { + throw s + } + } finally { + GlutenWriterColumnarRules.injectSparkLocalProperty(spark, None) + } + } + committer.addedStatuses.toSeq ++ committer.changeFiles + } + private def shouldOptimizeWrite( writeOptions: Option[DeltaOptions], sessionConf: SQLConf): Boolean = { @@ -188,17 +206,23 @@ class ClickhouseOptimisticTransaction( } override protected def getCommitter(outputPath: Path): DelayedCommitProtocol = - new CHDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir) + new FileDelayedCommitProtocol("delta", outputPath.toString, None, deltaDataSubdir) + + private def getCommitter2(outputPath: Path): DelayedCommitProtocol = { + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + new MergeTreeDelayedCommitProtocol2( + outputPath.toString, + None, + deltaDataSubdir, + tableV2.dataBaseName, + tableV2.tableName) + } - override def writeFiles( + private def pipelineWriteFiles( inputData: Dataset[_], writeOptions: Option[DeltaOptions], isOptimize: Boolean, additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - - if (isOptimize) - throw new UnsupportedOperationException("Optimize is not supported for ClickHouse") - hasWritten = true val spark = inputData.sparkSession @@ -230,14 +254,19 @@ class ClickhouseOptimisticTransaction( WriteFiles(logicalPlan, fileFormat, partitioningColumns, None, options, Map.empty) val queryExecution = new QueryExecution(spark, write) - val committer = getCommitter(outputPath) + val (committer, collectStats) = fileFormat.toString match { + case "MergeTree" => (getCommitter2(outputPath), false) + case _ => (getCommitter(outputPath), true) + } // If Statistics Collection is enabled, then create a stats tracker that will be injected during // the FileFormatWriter.write call below and will collect per-file stats using // StatisticsCollection - // val (optionalStatsTracker, _) = - // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) - val optionalStatsTracker: Option[DeltaJobStatisticsTracker] = None + val (optionalStatsTracker, _) = if (collectStats) { + getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) + } else { + (None, None) + } val constraints = Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints @@ -258,7 +287,7 @@ class ClickhouseOptimisticTransaction( DeltaOptimizedWriterExec(checkInvariants, metadata.partitionColumns, deltaLog) } else { checkInvariants - }*/ + } */ val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { @@ -304,10 +333,11 @@ class ClickhouseOptimisticTransaction( committer.addedStatuses }) .filter { - // In some cases, we can write out an empty `inputData`. Some examples of this (though, they - // may be fixed in the future) are the MERGE command when you delete with empty source, or - // empty target, or on disjoint tables. This is hard to catch before the write without - // collecting the DF ahead of time. Instead, we can return only the AddFiles that + // In some cases, we can write out an empty `inputData`. Some examples of this (though, + // they may be fixed in the future) are the MERGE command when you delete with empty + // source, or empty target, or on disjoint tables. This is hard to catch before + // the write without collecting the DF ahead of time. Instead, + // we can return only the AddFiles that // a) actually add rows, or // b) don't have any stats so we don't know the number of rows at all case a: AddFile => a.numLogicalRecords.forall(_ > 0) diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala index bac5231309b8..f64de28f4214 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/DeltaLog.scala @@ -60,7 +60,7 @@ import org.apache.spark.util._ /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0, it is modified to overcome the following issues: + * This file is copied from Delta 3.2.1, it is modified to overcome the following issues: * 1. return ClickhouseOptimisticTransaction * 2. return DeltaMergeTreeFileFormat * 3. create HadoopFsRelation with the bucket options diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala index 8836f7c88d23..5bfda914db67 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/Snapshot.scala @@ -45,7 +45,7 @@ import org.apache.spark.util.Utils /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. filesForScan() will cache the DeltaScan by the FilterExprsAsKey * 2. filesForScan() should return DeltaScan of AddMergeTreeParts instead of AddFile */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala index 5f6a2dc3d712..d887e7a21b34 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.V1Table import org.apache.spark.sql.connector.read.InputPartition -import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaErrors, DeltaLog, DeltaTableUtils, DeltaTimeTravelSpec, Snapshot, UnresolvedPathBasedDeltaTable} import org.apache.spark.sql.delta.actions.{Metadata, Protocol} import org.apache.spark.sql.delta.sources.DeltaDataSource @@ -89,13 +88,6 @@ class ClickHouseTableV2( ret } - override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { - new WriteIntoDeltaBuilder( - this, - info.options, - spark.sessionState.conf.useNullsForMissingDefaultColumnValues) - } - def getFileFormat(protocol: Protocol, meta: Metadata): DeltaMergeTreeFileFormat = { new DeltaMergeTreeFileFormat( protocol, diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala index dec1f4b9c3f5..0a25346fc6c3 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/DeleteCommand.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index 5b2170220228..439111df1b1c 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -49,7 +49,7 @@ import org.apache.spark.util.{SystemClock, ThreadUtils} /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified in: + * This file is copied from Delta 3.2.1. It is modified in: * 1. getDeltaTable supports to get ClickHouseTableV2 * 2. runOptimizeBinJobClickhouse * 3. groupFilesIntoBinsClickhouse diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala index 9a7fb96775f0..4e75b8461970 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/UpdateCommand.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.LongType /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala index 5d05bdb86896..7a350ae4d594 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -44,7 +44,7 @@ import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock} /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Gluten, part is a directory, but VacuumCommand assumes part is a file. So we need some * modifications to make it work. * 2. Set the 'gluten.enabledForCurrentThread' to false, now gluten can not support vacuum cmd. @@ -255,7 +255,8 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { val originalEnabledGluten = spark.sparkContext.getLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY) // gluten can not support vacuum command - spark.sparkContext.setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") + spark.sparkContext + .setLocalProperty(GlutenSessionExtensions.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") // --- modified end val validFiles = diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala index 42a89d427197..aa1f94c5c9f5 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/commands/merge/ClassicMergeExecutor.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.functions.{coalesce, col, count, input_file_name, li /** * Gluten overwrite Delta: * - * This file is copied from Delta 3.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 3.2.1. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). In findTouchedFiles func. */ diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala index f2c22234a692..e90df0f02634 100644 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/files/MergeTreeDelayedCommitProtocol.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.delta.files +import org.apache.hadoop.mapreduce.TaskAttemptContext + class MergeTreeDelayedCommitProtocol( val outputPath: String, randomPrefixLength: Option[Int], @@ -24,3 +26,49 @@ class MergeTreeDelayedCommitProtocol( val tableName: String) extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir) with MergeTreeFileCommitProtocol {} + +/** + * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline + * write for parquet. + */ +class FileDelayedCommitProtocol( + jobId: String, + val outputPath: String, + randomPrefixLength: Option[Int], + subdir: Option[String]) + extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) { + + override def getFileName( + taskContext: TaskAttemptContext, + ext: String, + partitionValues: Map[String, String]): String = { + super.getFileName(taskContext, ext, partitionValues) + } + + def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = { + assert(addedFiles.isEmpty) + addedFiles ++= files + } + + override def parsePartitions(dir: String): Map[String, String] = + super.parsePartitions(dir) +} + +/** + * A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields in a pipeline + * write for mergetree. + */ +class MergeTreeDelayedCommitProtocol2( + val outputPath: String, + randomPrefixLength: Option[Int], + subdir: Option[String], + val database: String, + val tableName: String) + extends DelayedCommitProtocol("delta-mergetree", outputPath, randomPrefixLength, subdir) { + + override def newTaskTempFile( + taskContext: TaskAttemptContext, + dir: Option[String], + ext: String): String = outputPath + +} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala new file mode 100644 index 000000000000..dbb5c4050a2c --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/delta/rules/CHOptimizeMetadataOnlyDeltaQuery.scala @@ -0,0 +1,77 @@ +/* + * 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.delta.rules + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, V2WriteCommand} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.{OptimisticTransaction, Snapshot, SubqueryTransformerHelper} +import org.apache.spark.sql.delta.files.TahoeLogFileIndex +import org.apache.spark.sql.delta.metering.DeltaLogging +import org.apache.spark.sql.delta.perf.OptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.stats.DeltaScanGenerator + +import org.apache.hadoop.fs.Path + +class CHOptimizeMetadataOnlyDeltaQuery(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with DeltaLogging + with SubqueryTransformerHelper + with OptimizeMetadataOnlyDeltaQuery { + + private val scannedSnapshots = + new java.util.concurrent.ConcurrentHashMap[(String, Path), Snapshot] + + protected def getDeltaScanGenerator(index: TahoeLogFileIndex): DeltaScanGenerator = { + // The first case means that we've fixed the table snapshot for time travel + if (index.isTimeTravelQuery) return index.getSnapshot + OptimisticTransaction + .getActive() + .map(_.getDeltaScanGenerator(index)) + .getOrElse { + // Will be called only when the log is accessed the first time + scannedSnapshots.computeIfAbsent(index.deltaLog.compositeId, _ => index.getSnapshot) + } + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + // Should not be applied to subqueries to avoid duplicate delta jobs. + val isSubquery = isSubqueryRoot(plan) + // Should not be applied to DataSourceV2 write plans, because they'll be planned later + // through a V1 fallback and only that later planning takes place within the transaction. + val isDataSourceV2 = plan.isInstanceOf[V2WriteCommand] + if (isSubquery || isDataSourceV2) { + return plan + } + // when 'stats.skipping' is off, it still use the metadata to optimize query for count/min/max + if ( + spark.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE, + CHBackendSettings.GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE) + .toBoolean && + !spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_STATS_SKIPPING, true) + ) { + optimizeQueryWithMetadata(plan) + } else { + plan + } + } +} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala deleted file mode 100644 index 66f502038fcd..000000000000 --- a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/CHDeltaColumnarWrite.scala +++ /dev/null @@ -1,144 +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. - */ -package org.apache.spark.sql.execution - -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException - -import org.apache.spark.internal.Logging -import org.apache.spark.internal.io.FileCommitProtocol -import org.apache.spark.sql.delta.files.DelayedCommitProtocol -import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.Utils - -import org.apache.hadoop.mapreduce.TaskAttemptContext - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer - -/** A Wrapper of [[DelayedCommitProtocol]] for accessing protected methods and fields. */ -class CHDelayedCommitProtocol( - jobId: String, - val outputPath: String, - randomPrefixLength: Option[Int], - subdir: Option[String]) - extends DelayedCommitProtocol(jobId, outputPath, randomPrefixLength, subdir) { - - override def getFileName( - taskContext: TaskAttemptContext, - ext: String, - partitionValues: Map[String, String]): String = { - super.getFileName(taskContext, ext, partitionValues) - } - - def updateAddedFiles(files: Seq[(Map[String, String], String)]): Unit = { - assert(addedFiles.isEmpty) - addedFiles ++= files - } - - override def parsePartitions(dir: String): Map[String, String] = - super.parsePartitions(dir) -} - -case class CHDelayedCommitProtocolWrite( - override val jobTrackerID: String, - override val description: WriteJobDescription, - override val committer: CHDelayedCommitProtocol) - extends CHColumnarWrite[CHDelayedCommitProtocol] - with Logging { - - override def doSetupNativeTask(): Unit = { - assert(description.path == committer.outputPath) - val nameSpec = CreateFileNameSpec(taskAttemptContext, description) - val writePath = description.path - val writeFileName = committer.getFileName(taskAttemptContext, nameSpec.suffix, Map.empty) - logDebug(s"Native staging write path: $writePath and file name: $writeFileName") - BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName) - } - - private def doCollectNativeResult( - cb: ColumnarBatch): Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = { - val numFiles = cb.numRows() - // Write an empty iterator - if (numFiles == 0) { - None - } else { - val file_col = cb.column(0) - val partition_col = cb.column(1) - val count_col = cb.column(2) - - val partitions: mutable.Set[String] = mutable.Set[String]() - val addedFiles: ArrayBuffer[(Map[String, String], String)] = - new ArrayBuffer[(Map[String, String], String)] - - var numWrittenRows: Long = 0 - Range(0, cb.numRows()).foreach { - i => - val fileName = file_col.getUTF8String(i).toString - val partition = partition_col.getUTF8String(i).toString - if (partition == "__NO_PARTITION_ID__") { - addedFiles.append((Map.empty[String, String], fileName)) - } else { - val partitionValues = committer.parsePartitions(partition) - addedFiles.append((partitionValues, s"$partition/$fileName")) - } - numWrittenRows += count_col.getLong(i) - } - val updatedPartitions = partitions.toSet - Some( - ( - addedFiles.toSeq, - ExecutedWriteSummary( - updatedPartitions = updatedPartitions, - stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows))))) - } - } - - override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { - doCollectNativeResult(batch).map { - case (addedFiles, summary) => - require(addedFiles.nonEmpty, "No files to commit") - - committer.updateAddedFiles(addedFiles) - - val (taskCommitMessage, taskCommitTime) = Utils.timeTakenMs { - committer.commitTask(taskAttemptContext) - } - - // Just for update task commit time - description.statsTrackers.foreach { - stats => stats.newTaskInstance().getFinalStats(taskCommitTime) - } - WriteTaskResult(taskCommitMessage, summary) - } - } -} - -object CHDeltaColumnarWrite { - def apply( - jobTrackerID: String, - description: WriteJobDescription, - committer: FileCommitProtocol): CHColumnarWrite[FileCommitProtocol] = committer match { - case c: CHDelayedCommitProtocol => - CHDelayedCommitProtocolWrite(jobTrackerID, description, c) - .asInstanceOf[CHColumnarWrite[FileCommitProtocol]] - case _ => - throw new GlutenNotSupportException( - s"Unsupported committer type: ${committer.getClass.getSimpleName}") - } -} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala new file mode 100644 index 000000000000..bf6b0c0074dc --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/FileDeltaColumnarWrite.scala @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.vectorized.NativeExpressionEvaluator + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Projection, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, DeclarativeAggregate} +import org.apache.spark.sql.delta.files.{FileDelayedCommitProtocol, MergeTreeDelayedCommitProtocol2} +import org.apache.spark.sql.delta.stats.DeltaFileStatistics +import org.apache.spark.sql.execution.datasources.{ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.types.StringType +import org.apache.spark.util.Utils + +import scala.collection.mutable.ArrayBuffer + +case class DeltaFileCommitInfo(committer: FileDelayedCommitProtocol) + extends (NativeFileWriteResult => Unit) { + val addedFiles: ArrayBuffer[(Map[String, String], String)] = + new ArrayBuffer[(Map[String, String], String)] + override def apply(stat: NativeFileWriteResult): Unit = { + if (stat.partition_id == "__NO_PARTITION_ID__") { + addedFiles.append((Map.empty[String, String], stat.filename)) + } else { + val partitionValues = committer.parsePartitions(stat.partition_id) + addedFiles.append((partitionValues, stat.relativePath)) + } + } + + def result: Seq[(Map[String, String], String)] = addedFiles.toSeq +} + +case class NativeDeltaStats(projection: Projection) extends (InternalRow => Unit) { + protected val results = new collection.mutable.HashMap[String, String] + + override def apply(row: InternalRow): Unit = { + val filename = row.getString(0) + val jsonStats = projection(row).getString(0) + assert(!results.contains(filename), s"Duplicate filename: $filename") + results.put(filename, jsonStats) + } + + def result: DeltaFileStatistics = DeltaFileStatistics(results.toMap) +} +case class FileDeltaColumnarWrite( + override val jobTrackerID: String, + override val description: WriteJobDescription, + override val committer: FileDelayedCommitProtocol) + extends CHColumnarWrite[FileDelayedCommitProtocol] + with Logging { + + private lazy val nativeDeltaStats: Option[NativeDeltaStats] = { + deltaWriteJobStatsTracker + .map( + delta => { + val r = delta.statsColExpr.transform { + case ae: AggregateExpression + if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] => + ae.aggregateFunction.asInstanceOf[DeclarativeAggregate].evaluateExpression + } + val z = Seq( + AttributeReference("filename", StringType, nullable = false)(), + AttributeReference("partition_id", StringType, nullable = false)()) + val s = + delta.statsColExpr + .collect { + case ae: AggregateExpression + if ae.aggregateFunction.isInstanceOf[DeclarativeAggregate] => + ae.aggregateFunction.asInstanceOf[DeclarativeAggregate] + } + .asInstanceOf[Seq[DeclarativeAggregate]] + .flatMap(_.aggBufferAttributes) + NativeDeltaStats( + UnsafeProjection.create( + exprs = Seq(r), + inputSchema = z :++ s + )) + }) + } + override def doSetupNativeTask(): Unit = { + assert(description.path == committer.outputPath) + val nameSpec = CreateFileNameSpec(taskAttemptContext, description) + val writePath = description.path + val writeFileName = committer.getFileName(taskAttemptContext, nameSpec.suffix, Map.empty) + + /** + * CDC files (CDC_PARTITION_COL = true) are named with "cdc-..." instead of "part-...".So, using + * pattern match to replace guid to {}.See the following example: + * {{{ + * part-00000-7d672b28-c079-4b00-bb0a-196c15112918-c000.snappy.parquet + * => + * part-00000-{id}.snappy.parquet + * }}} + */ + val guidPattern = + """.*-([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12})(?:-c(\d+)\..*)?$""".r + val fileNamePattern = + guidPattern.replaceAllIn( + writeFileName, + m => writeFileName.replace(m.group(1), FileNamePlaceHolder.ID)) + + logDebug(s"Native staging write path: $writePath and with pattern: $fileNamePattern") + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> fileNamePattern + ) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) + } + + private def doCollectNativeResult(stats: Seq[InternalRow]) + : Option[(Seq[(Map[String, String], String)], ExecutedWriteSummary)] = { + + // Write an empty iterator + if (stats.isEmpty) { + None + } else { + // stats.map(row => x.apply(row).getString(0)).foreach(println) + // process stats + val commitInfo = DeltaFileCommitInfo(committer) + val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker) + val basicNativeStats = Seq(commitInfo, basicNativeStat) + NativeStatCompute(stats)(basicNativeStats, nativeDeltaStats) + + Some( + ( + commitInfo.result, + ExecutedWriteSummary( + updatedPartitions = Set.empty, + stats = nativeDeltaStats.map(_.result).toSeq ++ Seq(basicNativeStat.result)))) + } + } + + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults).map { + case (addedFiles, summary) => + require(addedFiles.nonEmpty, "No files to commit") + + committer.updateAddedFiles(addedFiles) + + val (taskCommitMessage, taskCommitTime) = Utils.timeTakenMs { + committer.commitTask(taskAttemptContext) + } + + // Just for update task commit time + description.statsTrackers.foreach { + stats => stats.newTaskInstance().getFinalStats(taskCommitTime) + } + WriteTaskResult(taskCommitMessage, summary) + } + } +} + +object CHDeltaColumnarWrite { + def apply( + jobTrackerID: String, + description: WriteJobDescription, + committer: FileCommitProtocol): CHColumnarWrite[FileCommitProtocol] = committer match { + case c: FileDelayedCommitProtocol => + FileDeltaColumnarWrite(jobTrackerID, description, c) + .asInstanceOf[CHColumnarWrite[FileCommitProtocol]] + case m: MergeTreeDelayedCommitProtocol2 => + MergeTreeDeltaColumnarWrite(jobTrackerID, description, m) + .asInstanceOf[CHColumnarWrite[FileCommitProtocol]] + case _ => + throw new GlutenNotSupportException( + s"Unsupported committer type: ${committer.getClass.getSimpleName}") + } +} diff --git a/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala new file mode 100644 index 000000000000..3ac9d4c305e7 --- /dev/null +++ b/backends-clickhouse/src/main/delta-32/org/apache/spark/sql/execution/MergeTreeDeltaColumnarWrite.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings +import org.apache.gluten.vectorized.NativeExpressionEvaluator + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.delta.actions.{AddFile, FileAction} +import org.apache.spark.sql.delta.files.MergeTreeDelayedCommitProtocol2 +import org.apache.spark.sql.delta.stats.DeltaStatistics +import org.apache.spark.sql.delta.util.{JsonUtils, MergeTreePartitionUtils} +import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.mergetree.StorageMeta +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path + +import java.util.UUID + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions + +/** + * {{{ + * val schema = + * StructType( + * StructField("part_name", StringType, false) :: + * StructField("partition_id", StringType, false) :: + * StructField("record_count", LongType, false) :: + * StructField("marks_count", LongType, false) :: + * StructField("size_in_bytes", LongType, false) :: Nil) + * }}} + */ +case class MergeTreeWriteResult( + part_name: String, + partition_id: String, + record_count: Long, + marks_count: Long, + size_in_bytes: Long) { + def apply( + database: String, + table: String, + path: Path, + modificationTime: Long, + hostName: Seq[String]): FileAction = { + val partitionValues = if (partition_id == "__NO_PARTITION_ID__") { + Map.empty[String, String] + } else { + MergeTreePartitionUtils.parsePartitions(partition_id) + } + val tags = Map[String, String]( + "database" -> database, + "table" -> table, + "engine" -> "MergeTree", + "path" -> path.toUri.getPath, + "targetNode" -> hostName.map(_.trim).mkString(","), + "partition" -> "", + "uuid" -> "", + "rows" -> record_count.toString, + "bytesOnDisk" -> size_in_bytes.toString, + "dataCompressedBytes" -> -1L.toString, + "dataUncompressedBytes" -> -1L.toString, + "modificationTime" -> modificationTime.toString, + "partitionId" -> "", + "minBlockNumber" -> -1L.toString, + "maxBlockNumber" -> -1L.toString, + "level" -> -1L.toString, + "dataVersion" -> -1L.toString, + "defaultCompressionCodec" -> "lz4", + "bucketNum" -> "", + "dirName" -> path.toString, + "marks" -> marks_count.toString + ) + + val stats = Map[String, Any]( + DeltaStatistics.NUM_RECORDS -> record_count, + DeltaStatistics.MIN -> "", + DeltaStatistics.MAX -> "", + DeltaStatistics.NULL_COUNT -> "" + ) + AddFile( + part_name, + partitionValues, + size_in_bytes, + modificationTime, + dataChange = true, + JsonUtils.toJson(stats), + tags) + } +} + +object MergeTreeWriteResult { + implicit def apply(row: InternalRow): MergeTreeWriteResult = MergeTreeWriteResult( + row.getString(0), + row.getString(1), + row.getLong(2), + row.getLong(3), + row.getLong(4)) +} + +case class MergeTreeCommitInfo(committer: MergeTreeDelayedCommitProtocol2) + extends (MergeTreeWriteResult => Unit) { + private val modificationTime = System.currentTimeMillis() + private val hostName = Seq(Utils.localHostName()) + private val addedFiles: ArrayBuffer[FileAction] = new ArrayBuffer[FileAction] + private val path = new Path(committer.outputPath) + def apply(stat: MergeTreeWriteResult): Unit = { + addedFiles.append( + stat(committer.database, committer.tableName, path, modificationTime, hostName)) + } + def result: Seq[FileAction] = addedFiles.toSeq +} + +case class MergeTreeBasicWriteTaskStatsTracker() extends (MergeTreeWriteResult => Unit) { + private val partitions: mutable.ArrayBuffer[InternalRow] = mutable.ArrayBuffer.empty + private var numRows: Long = 0 + private var numBytes: Long = 0 + private var numFiles: Int = 0 + + def apply(stat: MergeTreeWriteResult): Unit = { + if (stat.partition_id != "__NO_PARTITION_ID__") { + partitions.append(new GenericInternalRow(Array[Any](stat.partition_id))) + } + numFiles += 1 + numRows += stat.record_count + numBytes += stat.size_in_bytes + } + + def result: BasicWriteTaskStats = + BasicWriteTaskStats(partitions.toSeq, numFiles, numBytes, numRows) +} + +case class MergeTreeDeltaColumnarWrite( + override val jobTrackerID: String, + override val description: WriteJobDescription, + override val committer: MergeTreeDelayedCommitProtocol2) + extends CHColumnarWrite[MergeTreeDelayedCommitProtocol2] + with Logging { + override def doSetupNativeTask(): Unit = { + assert(description.path == committer.outputPath) + val writePath = StorageMeta.normalizeRelativePath(committer.outputPath) + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val partPrefixWithoutPartitionAndBucket = s"${UUID.randomUUID.toString}_$split" + logDebug( + s"Pipeline write path: $writePath with part prefix: $partPrefixWithoutPartitionAndBucket") + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.PART_NAME_PREFIX.key -> partPrefixWithoutPartitionAndBucket) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) + } + + private def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = { + if (stats.isEmpty) { + None + } else { + val commitInfo = MergeTreeCommitInfo(committer) + val mergeTreeStat = MergeTreeBasicWriteTaskStatsTracker() + val basicNativeStats = Seq(commitInfo, mergeTreeStat) + NativeStatCompute(stats)(basicNativeStats) + + Some { + WriteTaskResult( + new TaskCommitMessage(commitInfo.result), + ExecutedWriteSummary(updatedPartitions = Set.empty, stats = Seq(mergeTreeStat.result)) + ) + } + } + } + + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults) + } +} diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java index 119dc61893d9..36f7c388baee 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/memory/CHThreadGroup.java @@ -28,7 +28,13 @@ public class CHThreadGroup implements TaskResource { */ public static void registerNewThreadGroup() { if (TaskResources.isResourceRegistered(CHThreadGroup.class.getName())) return; - CHThreadGroup group = new CHThreadGroup(); + + String taskId = ""; + if (TaskResources.getLocalTaskContext() != null) { + taskId = String.valueOf(TaskResources.getLocalTaskContext().taskAttemptId()); + } + + CHThreadGroup group = new CHThreadGroup(taskId); TaskResources.addResource(CHThreadGroup.class.getName(), group); TaskContext.get() .addTaskCompletionListener( @@ -40,8 +46,8 @@ public static void registerNewThreadGroup() { private long thread_group_id = 0; private long peak_memory = -1; - private CHThreadGroup() { - thread_group_id = createThreadGroup(); + private CHThreadGroup(String taskId) { + thread_group_id = createThreadGroup(taskId); } public long getPeakMemory() { @@ -69,7 +75,7 @@ public String resourceName() { return "CHThreadGroup"; } - private static native long createThreadGroup(); + private static native long createThreadGroup(String taskId); private static native long threadGroupPeakMemory(long id); diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java index a4f272a85355..2099ddbbf545 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/CHNativeExpressionEvaluator.java @@ -17,7 +17,6 @@ package org.apache.gluten.vectorized; import org.apache.gluten.GlutenConfig; -import org.apache.gluten.backend.Backend; import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.execution.ColumnarNativeIterator; import org.apache.gluten.memory.CHThreadGroup; @@ -25,7 +24,6 @@ import org.apache.spark.sql.internal.SQLConf; -import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; @@ -36,11 +34,12 @@ private CHNativeExpressionEvaluator() {} // Used to initialize the native computing. public static void initNative(scala.collection.Map conf) { Map nativeConfMap = - GlutenConfig.getNativeBackendConf(Backend.get().name(), conf); + GlutenConfig.getNativeBackendConf(BackendsApiManager.getBackendName(), conf); // Get the customer config from SparkConf for each backend BackendsApiManager.getTransformerApiInstance() - .postProcessNativeConfig(nativeConfMap, GlutenConfig.prefixOf(Backend.get().name())); + .postProcessNativeConfig( + nativeConfMap, GlutenConfig.prefixOf(BackendsApiManager.getBackendName())); nativeInitNative(ConfigUtil.serialize(nativeConfMap)); } @@ -55,12 +54,8 @@ public static boolean doValidate(byte[] subPlan) { } private static Map getNativeBackendConf() { - return GlutenConfig.getNativeBackendConf(Backend.get().name(), SQLConf.get().getAllConfs()); - } - - public static void injectWriteFilesTempPath(String path, String fileName) { - ExpressionEvaluatorJniWrapper.injectWriteFilesTempPath( - path.getBytes(StandardCharsets.UTF_8), fileName.getBytes(StandardCharsets.UTF_8)); + return GlutenConfig.getNativeBackendConf( + BackendsApiManager.getBackendName(), SQLConf.get().getAllConfs()); } // Used by WholeStageTransform to create the native computing pipeline and diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java index 8cc3046436ab..76fddcafdd41 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/ExpressionEvaluatorJniWrapper.java @@ -16,12 +16,7 @@ */ package org.apache.gluten.vectorized; -import org.apache.gluten.backendsapi.clickhouse.CHConf; import org.apache.gluten.execution.ColumnarNativeIterator; -import org.apache.gluten.utils.ConfigUtil; - -import java.util.Map; -import java.util.stream.Collectors; /** * This class is implemented in JNI. This provides the Java interface to invoke functions in JNI. @@ -48,23 +43,5 @@ public static native long nativeCreateKernelWithIterator( byte[] confArray, boolean materializeInput); - /** - * Set the temp path for writing files. - * - * @param path the temp path for writing files - */ - public static native void injectWriteFilesTempPath(byte[] path, byte[] filename); - - /// The following methods are used to update the query settings in the native engine. - public static void updateQueryRuntimeSettings(Map settings) { - Map newSettings = - settings.entrySet().stream() - .filter(entry -> CHConf.startWithSettingsPrefix(entry.getKey())) - .collect( - Collectors.toMap( - e -> CHConf.removeSettingsPrefix(e.getKey()), Map.Entry::getValue)); - updateQueryRuntimeSettings(ConfigUtil.serialize(newSettings)); - } - - private static native void updateQueryRuntimeSettings(byte[] settings); + public static native void updateQueryRuntimeSettings(byte[] settings); } diff --git a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java index b09cccb4580f..c73e1076ddf8 100644 --- a/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java +++ b/backends-clickhouse/src/main/java/org/apache/gluten/vectorized/SimpleExpressionEval.java @@ -34,7 +34,7 @@ public class SimpleExpressionEval implements AutoCloseable, Iterator { public SimpleExpressionEval(ColumnarNativeIterator blockStream, PlanNode planNode) { Plan plan = planNode.toProtobuf(); if (LOG.isDebugEnabled()) { - LOG.debug(String.format("SimpleExpressionEval exec plan: %s", plan.toString())); + LOG.debug("SimpleExpressionEval exec plan: {}", plan.toString()); } byte[] planData = plan.toByteArray(); instance = createNativeInstance(blockStream, planData); diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java index fb2b50798836..ee000d5ae157 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/CHDatasourceJniWrapper.java @@ -26,10 +26,8 @@ public CHDatasourceJniWrapper(String filePath, WriteRel write) { this.instance = createFilerWriter(filePath, write.toByteArray()); } - public CHDatasourceJniWrapper( - String prefix, String partition, String bucket, WriteRel write, byte[] confArray) { - this.instance = - createMergeTreeWriter(prefix, partition, bucket, write.toByteArray(), confArray); + public CHDatasourceJniWrapper(WriteRel write, byte[] confArray) { + this.instance = createMergeTreeWriter(write.toByteArray(), confArray); } public void write(long blockAddress) { @@ -48,8 +46,7 @@ public void close() { private native long createFilerWriter(String filePath, byte[] writeRel); /// MergeTreeWriter - private native long createMergeTreeWriter( - String prefix, String partition, String bucket, byte[] writeRel, byte[] confArray); + private native long createMergeTreeWriter(byte[] writeRel, byte[] confArray); public static native String nativeMergeMTParts( byte[] splitInfo, String partition_dir, String bucket_dir); diff --git a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java index 223e92c28d1d..49311504ced0 100644 --- a/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java +++ b/backends-clickhouse/src/main/java/org/apache/spark/sql/execution/datasources/clickhouse/WriteReturnedMetric.java @@ -18,8 +18,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Map; - public class WriteReturnedMetric { @JsonProperty("part_name") @@ -38,7 +36,7 @@ public class WriteReturnedMetric { protected String bucketId; @JsonProperty("partition_values") - protected Map partitionValues; + protected String partitionValues; public String getPartName() { return partName; @@ -72,11 +70,11 @@ public void setRowCount(long rowCount) { this.rowCount = rowCount; } - public Map getPartitionValues() { + public String getPartitionValues() { return partitionValues; } - public void setPartitionValues(Map partitionValues) { + public void setPartitionValues(String partitionValues) { this.partitionValues = partitionValues; } diff --git a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto index 8404f81f19f4..89f606e4ffd3 100644 --- a/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto +++ b/backends-clickhouse/src/main/resources/org/apache/spark/sql/execution/datasources/v1/write_optimization.proto @@ -6,6 +6,8 @@ package local_engine; option java_package = "org.apache.spark.sql.execution.datasources.v1"; option java_multiple_files = true; +//TODO : set compression codec + message Write { message Common { string format = 1; diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala index 59d912d8e75d..83a92db51897 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHBackend.scala @@ -18,13 +18,13 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component.BuildInfo import org.apache.gluten.backendsapi._ import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.WindowFunctionsBuilder import org.apache.gluten.extension.ValidationResult -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat._ @@ -34,9 +34,8 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat @@ -48,10 +47,11 @@ import java.util.Locale import scala.util.control.Breaks.{break, breakable} class CHBackend extends SubstraitBackend { + import CHBackend._ override def name(): String = CHConf.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = CHBatch - override def buildInfo(): Backend.BuildInfo = - Backend.BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") + override def buildInfo(): BuildInfo = + BuildInfo("ClickHouse", CH_BRANCH, CH_COMMIT, "UNKNOWN") + override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() override def iteratorApi(): IteratorApi = new CHIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new CHSparkPlanExecApi override def transformerApi(): TransformerApi = new CHTransformerApi @@ -62,7 +62,17 @@ class CHBackend extends SubstraitBackend { override def settings(): BackendSettingsApi = CHBackendSettings } +object CHBackend { + private class ConvFunc() extends ConventionFunc.Override { + override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + CHBatch + } + } +} + object CHBackendSettings extends BackendSettingsApi with Logging { + override def primaryBatchType: Convention.BatchType = CHBatch private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD = "spark.gluten.sql.columnar.separate.scan.rdd.for.ch" private val GLUTEN_CLICKHOUSE_SEP_SCAN_RDD_DEFAULT = "false" @@ -133,6 +143,14 @@ object CHBackendSettings extends BackendSettingsApi with Logging { val GLUTEN_CLICKHOUSE_TABLE_PATH_TO_MTPS_CACHE_SIZE: String = CHConf.prefixOf("table.path.to.mtps.cache.size") + val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE: String = + CHConf.prefixOf("delta.metadata.optimize") + val GLUTEN_CLICKHOUSE_DELTA_METADATA_OPTIMIZE_DEFAULT_VALUE: String = "true" + + val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT: String = + CHConf.prefixOf("convert.left.anti_semi.to.right") + val GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE: String = "false" + def affinityMode: String = { SparkEnv.get.conf .get( @@ -142,10 +160,11 @@ object CHBackendSettings extends BackendSettingsApi with Logging { .toLowerCase(Locale.getDefault) } - override def validateScan( + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = { + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = { // Validate if all types are supported. def hasComplexType: Boolean = { @@ -227,45 +246,16 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } } - def validateBucketSpec(): Option[String] = { - if (bucketSpec.nonEmpty) { - Some("Unsupported native write: bucket write is not supported.") - } else { - None - } - } - validateCompressionCodec() .orElse(validateFileFormat()) .orElse(validateFieldMetadata()) .orElse(validateDateTypes()) - .orElse(validateWriteFilesOptions()) - .orElse(validateBucketSpec()) match { + .orElse(validateWriteFilesOptions()) match { case Some(reason) => ValidationResult.failed(reason) case _ => ValidationResult.succeeded } } - override def supportShuffleWithProject( - outputPartitioning: Partitioning, - child: SparkPlan): Boolean = { - child match { - case hash: HashAggregateExec => - if (hash.aggregateExpressions.isEmpty) { - true - } else { - outputPartitioning match { - case hashPartitioning: HashPartitioning => - hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference]) - case _ => - false - } - } - case _ => - true - } - } - override def supportSortExec(): Boolean = { GlutenConfig.getConf.enableColumnarSort } @@ -348,18 +338,47 @@ object CHBackendSettings extends BackendSettingsApi with Logging { ) } + // It try to move the expand node after the pre-aggregate node. That is to make the plan from + // expand -> pre-aggregate -> shuffle -> final-aggregate + // to + // pre-aggregate -> expand -> shuffle -> final-aggregate + // It could reduce the overhead of pre-aggregate node. + def enableLazyAggregateExpand(): Boolean = { + SparkEnv.get.conf.getBoolean( + CHConf.runtimeConfig("enable_lazy_aggregate_expand"), + defaultValue = true + ) + } + + // If the partition keys are high cardinality, the aggregation method is slower. + def enableConvertWindowGroupLimitToAggregate(): Boolean = { + SparkEnv.get.conf.getBoolean( + CHConf.runtimeConfig("enable_window_group_limit_to_aggregate"), + defaultValue = true + ) + } + override def enableNativeWriteFiles(): Boolean = { GlutenConfig.getConf.enableNativeWriter.getOrElse(false) } override def supportCartesianProductExec(): Boolean = true + override def supportCartesianProductExecWithCondition(): Boolean = false + override def supportHashBuildJoinTypeOnLeft: JoinType => Boolean = { t => if (super.supportHashBuildJoinTypeOnLeft(t)) { true } else { t match { + case LeftAnti | LeftSemi + if (SQLConf.get + .getConfString( + GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT, + GLUTEN_CLICKHOUSE_CONVERT_LEFT_ANTI_SEMI_TO_RIGHT_DEFAULT_VALUE) + .toBoolean) => + true case LeftOuter => true case _ => false } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala index 3f653ad200c0..83cc75bc0ac7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHConf.scala @@ -19,6 +19,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf object CHConf { private[clickhouse] val BACKEND_NAME: String = "ch" @@ -47,10 +48,26 @@ object CHConf { } } + /** CH configuration prefix at Java side */ def prefixOf(key: String): String = s"$CONF_PREFIX.$key" def runtimeConfig(key: String): String = s"$RUNTIME_CONFIG.$key" def runtimeSettings(key: String): String = s"$RUNTIME_SETTINGS.$key" def startWithSettingsPrefix(key: String): Boolean = key.startsWith(RUNTIME_SETTINGS) def removeSettingsPrefix(key: String): String = key.substring(RUNTIME_SETTINGS.length + 1) + + def get: CHConf = new CHConf(SQLConf.get) + + import SQLConf._ + + val ENABLE_ONEPIPELINE_MERGETREE_WRITE = + buildConf(prefixOf("mergetree.write.pipeline")) + .doc("Using one pipeline to write data to MergeTree table in Spark 3.5") + .booleanConf + .createWithDefault(false) +} + +class CHConf(conf: SQLConf) extends GlutenConfig(conf) { + def enableOnePipelineMergeTreeWrite: Boolean = + conf.getConf(CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE) } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala index 2fa2e4402b79..878e27a5b8c2 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHIteratorApi.scala @@ -21,7 +21,6 @@ import org.apache.gluten.backendsapi.IteratorApi import org.apache.gluten.execution._ import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.memory.CHThreadGroup import org.apache.gluten.metrics.IMetrics import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.plan.PlanNode @@ -44,7 +43,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration import java.lang.{Long => JLong} import java.net.URI @@ -134,8 +132,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case p: GlutenMergeTreePartition => ExtensionTableBuilder @@ -325,10 +322,6 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { createNativeIterator(splitInfoByteArray, wsPlan, materializeInput, inputIterators)) } - override def injectWriteFilesTempPath(path: String, fileName: String): Unit = { - CHThreadGroup.registerNewThreadGroup() - CHNativeExpressionEvaluator.injectWriteFilesTempPath(path, fileName) - } } class CollectMetricIterator( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala index 6ae957912ac5..b93c002561f0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHListenerApi.scala @@ -71,7 +71,7 @@ class CHListenerApi extends ListenerApi with Logging { private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. - CHBatch.getClass + CHBatch.ensureRegistered() SparkDirectoryUtil.init(conf) val libPath = conf.get(GlutenConfig.GLUTEN_LIB_PATH, StringUtils.EMPTY) if (StringUtils.isBlank(libPath)) { diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala index 6a4f0c9a6f77..a0576a807b98 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHMetricsApi.scala @@ -23,7 +23,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import java.lang.{Long => JLong} @@ -39,21 +40,40 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { } override def genInputIteratorTransformerMetrics( - sparkContext: SparkContext): Map[String, SQLMetric] = { + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] = { + def metricsPlan(plan: SparkPlan): SparkPlan = { + plan match { + case ColumnarInputAdapter(child) => metricsPlan(child) + case q: QueryStageExec => metricsPlan(q.plan) + case _ => plan + } + } + + val outputMetrics = if (forBroadcast) { + metricsPlan(child).metrics + .filterKeys(key => key.equals("numOutputRows")) + } else { + Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows") + ) + } + Map( "iterReadTime" -> SQLMetrics.createTimingMetric( sparkContext, "time of reading from iterator"), "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "fillingRightJoinSideTime" -> SQLMetrics.createTimingMetric( sparkContext, "filling right join side time") - ) + ) ++ outputMetrics } override def genInputIteratorTransformerMetricsUpdater( - metrics: Map[String, SQLMetric]): MetricsUpdater = { + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater = { InputIteratorMetricsUpdater(metrics) } @@ -430,6 +450,14 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { s"SampleTransformer metrics update is not supported in CH backend") } + override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = + throw new UnsupportedOperationException( + "UnionExecTransformer metrics update is not supported in CH backend") + + override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = + throw new UnsupportedOperationException( + "UnionExecTransformer metrics update is not supported in CH backend") + def genWriteFilesTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map( "physicalWrittenBytes" -> SQLMetrics.createMetric(sparkContext, "number of written bytes"), diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala index 91698d4cde85..141778688967 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHRuleApi.scala @@ -16,13 +16,19 @@ */ package org.apache.gluten.backendsapi.clickhouse +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi +import org.apache.gluten.columnarbatch.CHBatch import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} +import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.rewrite._ import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.parser.{GlutenCacheFilesSqlParser, GlutenClickhouseSqlParser} import org.apache.gluten.sql.shims.SparkShimLoader @@ -30,22 +36,23 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.catalyst.{CHAggregateFunctionRewriteRule, EqualToRewrite} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.delta.DeltaLogFileIndex -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, CommandResultExec, FileSourceScanExec, GlutenFallbackReporter, RDDScanExec, SparkPlan} +import org.apache.spark.sql.delta.rules.CHOptimizeMetadataOnlyDeltaQuery +import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.V2CommandExec import org.apache.spark.util.SparkPlanRules class CHRuleApi extends RuleApi { import CHRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) } } -private object CHRuleApi { - def injectSpark(injector: SparkInjector): Unit = { +object CHRuleApi { + private def injectSpark(injector: SparkInjector): Unit = { // Inject the regular Spark rules directly. injector.injectQueryStagePrepRule(FallbackBroadcastHashJoinPrepQueryStage.apply) injector.injectQueryStagePrepRule(spark => CHAQEPropagateEmptyRelation(spark)) @@ -56,38 +63,62 @@ private object CHRuleApi { injector.injectResolutionRule(spark => new RewriteToDateExpresstionRule(spark)) injector.injectResolutionRule(spark => new RewriteDateTimestampComparisonRule(spark)) injector.injectOptimizerRule(spark => new CommonSubexpressionEliminateRule(spark)) + injector.injectOptimizerRule(spark => new ExtendedColumnPruning(spark)) injector.injectOptimizerRule(spark => CHAggregateFunctionRewriteRule(spark)) injector.injectOptimizerRule(_ => CountDistinctWithoutExpand) injector.injectOptimizerRule(_ => EqualToRewrite) + injector.injectPreCBORule(spark => new CHOptimizeMetadataOnlyDeltaQuery(spark)) } - def injectLegacy(injector: LegacyInjector): Unit = { - - // Gluten columnar: Transform rules. - injector.injectTransform(_ => RemoveTransitions) - injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) - injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) - injector.injectTransform(c => FallbackMultiCodegens.apply(c.session)) - injector.injectTransform(_ => RewriteSubqueryBroadcast()) - injector.injectTransform(c => FallbackBroadcastHashJoin.apply(c.session)) - injector.injectTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) - injector.injectTransform(_ => intercept(RewriteSparkPlanRulesManager())) - injector.injectTransform(_ => intercept(AddFallbackTagRule())) - injector.injectTransform(_ => intercept(TransformPreOverrides())) - injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) - injector.injectTransform(c => intercept(RewriteTransformer.apply(c.session))) - injector.injectTransform(_ => PushDownFilterToScan) - injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) - injector.injectTransform(_ => EnsureLocalSortRequirements) - injector.injectTransform(_ => EliminateLocalSort) - injector.injectTransform(_ => CollapseProjectExecTransformer) - injector.injectTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) - injector.injectTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) + private def injectLegacy(injector: LegacyInjector): Unit = { + // Legacy: Pre-transform rules. + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => FallbackBroadcastHashJoin.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate.apply(c.session)) + injector.injectPreTransform(_ => WriteFilesWithBucketValue) + + // Legacy: The legacy transform rule. + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .fallbackByNativeValidation() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin()) injector.injectTransform( + c => intercept(HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads))) + + // Legacy: Post-transform rules. + injector.injectPostTransform(_ => PruneNestedColumnsInHiveTableScan) + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => intercept(RewriteTransformer.apply(c.session))) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => RewriteSortMergeJoinToHashJoinRule.apply(c.session)) + injector.injectPostTransform(c => PushdownAggregatePreProjectionAheadExpand.apply(c.session)) + injector.injectPostTransform(c => LazyAggregateExpandRule.apply(c.session)) + injector.injectPostTransform(c => ConverRowNumbertWindowToAggregateRule(c.session)) + injector.injectPostTransform( c => intercept( - SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarTransformRules)(c.session))) - injector.injectTransform(c => InsertTransitions(c.outputsColumnar)) + SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarTransformRules)( + c.session))) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, CHBatch)) + injector.injectPostTransform(c => RemoveDuplicatedColumns.apply(c.session)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -98,21 +129,22 @@ private object CHRuleApi { SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => intercept(each(c.session)))) - injector.injectPost(c => ColumnarCollapseTransformStages(c.conf)) - injector.injectTransform( + injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) + injector.injectPost( c => - intercept(SparkPlanRules.extendedColumnarRule(c.conf.extendedColumnarPostRules)(c.session))) + intercept( + SparkPlanRules.extendedColumnarRule(c.glutenConf.extendedColumnarPostRules)(c.session))) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session)) + injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.injectFinal(_ => RemoveFallbackTagRule()) } - def injectRas(injector: RasInjector): Unit = { + private def injectRas(injector: RasInjector): Unit = { // CH backend doesn't work with RAS at the moment. Inject a rule that aborts any // execution calls. - injector.inject( + injector.injectPreTransform( _ => new SparkPlanRules.AbortRule( "Clickhouse backend doesn't yet have RAS support, please try disabling RAS and" + diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala index ba165d936eed..de0680df10ab 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -23,8 +23,7 @@ import org.apache.gluten.execution._ import org.apache.gluten.expression._ import org.apache.gluten.expression.ExpressionNames.MONOTONICALLY_INCREASING_ID import org.apache.gluten.extension.ExpressionExtensionTrait -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.expression.{ExpressionBuilder, ExpressionNode, WindowFunctionNode} import org.apache.gluten.utils.{CHJoinValidateUtil, UnknownJoinStrategy} @@ -158,16 +157,21 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { aggregateAttributes: Seq[Attribute], initialInputBufferOffset: Int, resultExpressions: Seq[NamedExpression], - child: SparkPlan): HashAggregateExecBaseTransformer = + child: SparkPlan): HashAggregateExecBaseTransformer = { + val replacedResultExpressions = CHHashAggregateExecTransformer.getCHAggregateResultExpressions( + groupingExpressions, + aggregateExpressions, + resultExpressions) CHHashAggregateExecTransformer( requiredChildDistributionExpressions, - groupingExpressions.distinct, + groupingExpressions, aggregateExpressions, aggregateAttributes, initialInputBufferOffset, - resultExpressions.distinct, + replacedResultExpressions, child ) + } /** Generate HashAggregateExecPullOutHelper */ override def genHashAggregateExecPullOutHelper( @@ -219,9 +223,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform.static()( + ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newExprs = Seq[Expression]() for (i <- exprs.indices) { val pos = newExpressionsPosition(i) @@ -244,9 +249,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { } // FIXME: The operation happens inside ReplaceSingleNode(). // Caller may not know it adds project on top of the shuffle. - val project = TransformPreOverrides().apply( - AddFallbackTagRule().apply( - ProjectExec(plan.child.output ++ projectExpressions, plan.child))) + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val project = + HeuristicTransform.static()( + ProjectExec(plan.child.output ++ projectExpressions, plan.child)) var newOrderings = Seq[SortOrder]() for (i <- orderings.indices) { val oldOrdering = orderings(i) @@ -267,9 +273,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { override def genColumnarShuffleExchange(shuffle: ShuffleExchangeExec): SparkPlan = { val child = shuffle.child - if ( - BackendsApiManager.getSettings.supportShuffleWithProject(shuffle.outputPartitioning, child) - ) { + if (CHValidatorApi.supportShuffleWithProject(shuffle.outputPartitioning, child)) { val (projectColumnNumber, newPartitioning, newChild) = addProjectionForShuffleExchange(shuffle) @@ -359,15 +363,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi with Logging { left: SparkPlan, right: SparkPlan, condition: Option[Expression]): CartesianProductExecTransformer = - if (!condition.isEmpty) { - throw new GlutenNotSupportException( - "CartesianProductExecTransformer with condition is not supported in ch backend.") - } else { - CartesianProductExecTransformer( - ColumnarCartesianProductBridge(left), - ColumnarCartesianProductBridge(right), - condition) - } + CartesianProductExecTransformer( + ColumnarCartesianProductBridge(left), + ColumnarCartesianProductBridge(right), + condition) override def genBroadcastNestedLoopJoinExecTransformer( left: SparkPlan, diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index e5b71825853b..0be8cf2c25bf 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.backendsapi.TransformerApi -import org.apache.gluten.execution.{CHHashAggregateExecTransformer, WriteFilesExecTransformer} +import org.apache.gluten.execution.CHHashAggregateExecTransformer import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.substrait.expression.{BooleanLiteralNode, ExpressionBuilder, ExpressionNode} import org.apache.gluten.utils.{CHInputPartitionsUtil, ExpressionDocUtil} @@ -26,18 +26,23 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.GlutenDriverEndpoint import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.delta.MergeTreeFileFormat import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.datasources.{FileFormat, HadoopFsRelation, PartitionDirectory} +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.v1.Write import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet import com.google.common.collect.Lists -import com.google.protobuf.{Any, Message, StringValue} +import com.google.protobuf.{Any, Message} +import org.apache.hadoop.fs.Path import java.util @@ -177,10 +182,13 @@ class CHTransformerApi extends TransformerApi with Logging { // output name will be different from grouping expressions, // so using output attribute instead of grouping expression val groupingExpressions = hash.output.splitAt(hash.groupingExpressions.size)._1 - val aggResultAttributes = CHHashAggregateExecTransformer.getAggregateResultAttributes( - groupingExpressions, - hash.aggregateExpressions - ) + val aggResultAttributes = CHHashAggregateExecTransformer + .getCHAggregateResultExpressions( + groupingExpressions, + hash.aggregateExpressions, + hash.resultExpressions + ) + .map(_.toAttribute) if (aggResultAttributes.size == hash.output.size) { aggResultAttributes } else { @@ -243,16 +251,27 @@ class CHTransformerApi extends TransformerApi with Logging { register.shortName case _ => "UnknownFileFormat" } - val compressionCodec = - WriteFilesExecTransformer.getCompressionCodec(writeOptions).capitalize - val writeParametersStr = new StringBuffer("WriteParameters:") - writeParametersStr.append("is").append(compressionCodec).append("=1") - writeParametersStr.append(";format=").append(fileFormatStr).append("\n") + val write = Write + .newBuilder() + .setCommon( + Write.Common + .newBuilder() + .setFormat(fileFormatStr) + .setJobTaskAttemptId("") // we can get job and task id at the driver side + .build()) - packPBMessage( - StringValue - .newBuilder() - .setValue(writeParametersStr.toString) - .build()) + fileFormat match { + case d: MergeTreeFileFormat => + write.setMergetree(MergeTreeFileFormat.createWrite(d.metadata)) + case _: ParquetFileFormat => + write.setParquet(Write.ParquetWrite.newBuilder().build()) + case _: OrcFileFormat => + write.setOrc(Write.OrcWrite.newBuilder().build()) + } + packPBMessage(write.build()) } + + /** use Hadoop Path class to encode the file path */ + override def encodeFilePathIfNeed(filePath: String): String = + (new Path(filePath)).toUri.toASCIIString } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala index eed493cffe1e..29d26410b9b0 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHValidatorApi.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi.clickhouse import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi} +import org.apache.gluten.backendsapi.ValidatorApi import org.apache.gluten.expression.ExpressionConverter import org.apache.gluten.extension.ValidationResult import org.apache.gluten.substrait.SubstraitContext @@ -28,12 +28,14 @@ import org.apache.gluten.vectorized.CHNativeExpressionEvaluator import org.apache.spark.internal.Logging import org.apache.spark.shuffle.utils.RangePartitionerBoundsGenerator -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.aggregate.HashAggregateExec class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logging { + import CHValidatorApi._ override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = { if (CHNativeExpressionEvaluator.doValidate(plan.toProtobuf.toByteArray)) { @@ -86,10 +88,7 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg .doTransform(substraitContext.registeredFunction) node.isInstanceOf[SelectionNode] } - if ( - allSelectionNodes || - BackendsApiManager.getSettings.supportShuffleWithProject(outputPartitioning, child) - ) { + if (allSelectionNodes || supportShuffleWithProject(outputPartitioning, child)) { None } else { Some("expressions are not supported in HashPartitioning") @@ -107,3 +106,31 @@ class CHValidatorApi extends ValidatorApi with AdaptiveSparkPlanHelper with Logg } } } + +object CHValidatorApi { + + /** + * A shuffle key may be an expression. We would add a projection for this expression shuffle key + * and make it into a new column which the shuffle will refer to. But we need to remove it from + * the result columns from the shuffle. + * + * Since https://github.com/apache/incubator-gluten/pull/1071. + */ + def supportShuffleWithProject(outputPartitioning: Partitioning, child: SparkPlan): Boolean = { + child match { + case hash: HashAggregateExec => + if (hash.aggregateExpressions.isEmpty) { + true + } else { + outputPartitioning match { + case hashPartitioning: HashPartitioning => + hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference]) + case _ => + false + } + } + case _ => + true + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala new file mode 100644 index 000000000000..12bb8d05d953 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeConfig.scala @@ -0,0 +1,45 @@ +/* + * 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.gluten.backendsapi.clickhouse + +import org.apache.spark.sql.internal.SQLConf + +object RuntimeConfig { + import CHConf._ + import SQLConf._ + + val PATH = + buildConf(runtimeConfig("path")) + .doc( + "https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#path") + .stringConf + .createWithDefault("/") + + // scalastyle:off line.size.limit + val TMP_PATH = + buildConf(runtimeConfig("tmp_path")) + .doc("https://clickhouse.com/docs/en/operations/server-configuration-parameters/settings#tmp-path") + .stringConf + .createWithDefault("/tmp/libch") + // scalastyle:on line.size.limit + + val USE_CURRENT_DIRECTORY_AS_TMP = + buildConf(runtimeConfig("use_current_directory_as_tmp")) + .doc("Use the current directory as the temporary directory.") + .booleanConf + .createWithDefault(false) +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala new file mode 100644 index 000000000000..c2747cf1eb53 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/RuntimeSettings.scala @@ -0,0 +1,61 @@ +/* + * 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.gluten.backendsapi.clickhouse + +import org.apache.spark.sql.internal.SQLConf + +object RuntimeSettings { + + import CHConf._ + import SQLConf._ + + val NATIVE_WRITE_RESERVE_PARTITION_COLUMNS = + buildConf(runtimeSettings("gluten.write.reserve_partition_columns")) + .doc("Whether reserve partition columns for Native write or not, default is false") + .booleanConf + .createWithDefault(false) + + val TASK_WRITE_TMP_DIR = + buildConf(runtimeSettings("gluten.task_write_tmp_dir")) + .doc("The temporary directory for writing data") + .stringConf + .createWithDefault("") + + val TASK_WRITE_FILENAME_PATTERN = + buildConf(runtimeSettings("gluten.task_write_filename_pattern")) + .doc("The pattern to generate file name for writing delta parquet in spark 3.5") + .stringConf + .createWithDefault("") + + val PART_NAME_PREFIX = + buildConf(runtimeSettings("gluten.part_name_prefix")) + .doc("The part name prefix for writing data") + .stringConf + .createWithDefault("") + + val PARTITION_DIR = + buildConf(runtimeSettings("gluten.partition_dir")) + .doc("The partition directory for writing data") + .stringConf + .createWithDefault("") + + val BUCKET_DIR = + buildConf(runtimeSettings("gluten.bucket_dir")) + .doc("The bucket directory for writing data") + .stringConf + .createWithDefault("") +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala index 870a731b1176..ac0ca5f8b4b7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/columnarbatch/CHBatch.scala @@ -38,6 +38,8 @@ import org.apache.spark.sql.execution.{CHColumnarToRowExec, RowToCHNativeColumna * }}} */ object CHBatch extends Convention.BatchType { - fromRow(RowToCHNativeColumnarExec.apply) - toRow(CHColumnarToRowExec.apply) + override protected def registerTransitions(): Unit = { + fromRow(RowToCHNativeColumnarExec.apply) + toRow(CHColumnarToRowExec.apply) + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala new file mode 100644 index 000000000000..83bb33bfa225 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHAggregateGroupLimitExecTransformer.scala @@ -0,0 +1,174 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.exception.GlutenNotSupportException +import org.apache.gluten.expression._ +import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.metrics.MetricsUpdater +import org.apache.gluten.substrait.`type`.TypeBuilder +import org.apache.gluten.substrait.SubstraitContext +import org.apache.gluten.substrait.extensions.ExtensionBuilder +import org.apache.gluten.substrait.rel.{RelBuilder, RelNode} + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.SparkPlan + +import com.google.protobuf.StringValue +import io.substrait.proto.SortField + +import scala.collection.JavaConverters._ + +case class CHAggregateGroupLimitExecTransformer( + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + rankLikeFunction: Expression, + resultAttributes: Seq[Attribute], + limit: Int, + child: SparkPlan) + extends UnaryTransformSupport { + + @transient override lazy val metrics = + BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext) + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = + copy(child = newChild) + + override def metricsUpdater(): MetricsUpdater = + BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) + + override def output: Seq[Attribute] = resultAttributes + + override def requiredChildDistribution: Seq[Distribution] = { + if (partitionSpec.isEmpty) { + // Only show warning when the number of bytes is larger than 100 MiB? + logWarning( + "No Partition Defined for Window operation! Moving all data to a single " + + "partition, this can cause serious performance degradation.") + AllTuples :: Nil + } else ClusteredDistribution(partitionSpec) :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + Seq(Nil) + } + + override def outputOrdering: Seq[SortOrder] = { + if (requiredChildOrdering.forall(_.isEmpty)) { + Nil + } else { + child.outputOrdering + } + } + + override def outputPartitioning: Partitioning = child.outputPartitioning + + def getWindowGroupLimitRel( + context: SubstraitContext, + originalInputAttributes: Seq[Attribute], + operatorId: Long, + input: RelNode, + validation: Boolean): RelNode = { + val args = context.registeredFunction + // Partition By Expressions + val partitionsExpressions = partitionSpec + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, attributeSeq = child.output) + .doTransform(args)) + .asJava + + // Sort By Expressions + val sortFieldList = + orderSpec.map { + order => + val builder = SortField.newBuilder() + val exprNode = ExpressionConverter + .replaceWithExpressionTransformer(order.child, attributeSeq = child.output) + .doTransform(args) + builder.setExpr(exprNode.toProtobuf) + builder.setDirectionValue(SortExecTransformer.transformSortDirection(order)) + builder.build() + }.asJava + if (!validation) { + val windowFunction = rankLikeFunction match { + case _: RowNumber => ExpressionNames.ROW_NUMBER + case _: Rank => ExpressionNames.RANK + case _: DenseRank => ExpressionNames.DENSE_RANK + case _ => throw new GlutenNotSupportException(s"Unknow window function $rankLikeFunction") + } + val parametersStr = new StringBuffer("WindowGroupLimitParameters:") + parametersStr + .append("window_function=") + .append(windowFunction) + .append("\n") + .append("is_aggregate_group_limit=true\n") + val message = StringValue.newBuilder().setValue(parametersStr.toString).build() + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage(message), + null) + RelBuilder.makeWindowGroupLimitRel( + input, + partitionsExpressions, + sortFieldList, + limit, + extensionNode, + context, + operatorId) + } else { + // Use a extension node to send the input types through Substrait plan for validation. + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage( + TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) + + RelBuilder.makeWindowGroupLimitRel( + input, + partitionsExpressions, + sortFieldList, + limit, + extensionNode, + context, + operatorId) + } + } + + override protected def doValidateInternal(): ValidationResult = { + val substraitContext = new SubstraitContext + val operatorId = substraitContext.nextOperatorId(this.nodeName) + + val relNode = + getWindowGroupLimitRel(substraitContext, child.output, operatorId, null, validation = true) + + doNativeValidation(substraitContext, relNode) + } + + override protected def doTransform(context: SubstraitContext): TransformContext = { + val childCtx = child.asInstanceOf[TransformSupport].transform(context) + val operatorId = context.nextOperatorId(this.nodeName) + + val currRel = + getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) + assert(currRel != null, "Window Group Limit Rel should be valid") + TransformContext(output, currRel) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala index f5e64330cd15..48b0d7336103 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashAggregateExecTransformer.scala @@ -43,6 +43,45 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer object CHHashAggregateExecTransformer { + // The result attributes of aggregate expressions from vanilla may be different from CH native. + // For example, the result attributes of `avg(x)` are `sum(x)` and `count(x)`. This could bring + // some unexpected issues. So we need to make the result attributes consistent with CH native. + def getCHAggregateResultExpressions( + groupingExpressions: Seq[NamedExpression], + aggregateExpressions: Seq[AggregateExpression], + resultExpressions: Seq[NamedExpression]): Seq[NamedExpression] = { + var adjustedResultExpressions = resultExpressions.slice(0, groupingExpressions.length) + var resultExpressionIndex = groupingExpressions.length + adjustedResultExpressions ++ aggregateExpressions.flatMap { + aggExpr => + aggExpr.mode match { + case Partial | PartialMerge => + // For partial aggregate, the size of the result expressions of an aggregate expression + // is the same as aggBufferAttributes' length + val aggBufferAttributesCount = aggExpr.aggregateFunction.aggBufferAttributes.length + aggExpr.aggregateFunction match { + case avg: Average => + val res = Seq(aggExpr.resultAttribute) + resultExpressionIndex += aggBufferAttributesCount + res + case sum: Sum if (sum.dataType.isInstanceOf[DecimalType]) => + val res = Seq(resultExpressions(resultExpressionIndex)) + resultExpressionIndex += aggBufferAttributesCount + res + case _ => + val res = resultExpressions + .slice(resultExpressionIndex, resultExpressionIndex + aggBufferAttributesCount) + resultExpressionIndex += aggBufferAttributesCount + res + } + case _ => + val res = Seq(resultExpressions(resultExpressionIndex)) + resultExpressionIndex += 1 + res + } + } + } + def getAggregateResultAttributes( groupingExpressions: Seq[NamedExpression], aggregateExpressions: Seq[AggregateExpression]): Seq[Attribute] = { @@ -170,7 +209,7 @@ case class CHHashAggregateExecTransformer( RelBuilder.makeReadRelForInputIteratorWithoutRegister(typeList, nameList, context) (getAggRel(context, operatorId, aggParams, readRel), inputAttrs, outputAttrs) } - TransformContext(inputAttributes, outputAttributes, relNode) + TransformContext(outputAttributes, relNode) } override def getAggRel( diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala index 43f19c30e28f..6bf2248ebe3a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHHashJoinExecTransformer.scala @@ -58,16 +58,23 @@ object JoinTypeTransform { } else { JoinRel.JoinType.JOIN_TYPE_RIGHT } - case LeftSemi | ExistenceJoin(_) => + case LeftSemi => if (!buildRight) { - throw new IllegalArgumentException("LeftSemi join should not switch children") + JoinRel.JoinType.JOIN_TYPE_RIGHT_SEMI + } else { + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI } - JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI case LeftAnti => if (!buildRight) { - throw new IllegalArgumentException("LeftAnti join should not switch children") + JoinRel.JoinType.JOIN_TYPE_RIGHT_ANTI + } else { + JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI + } + case ExistenceJoin(_) => + if (!buildRight) { + throw new IllegalArgumentException("Existence join should not switch children") } - JoinRel.JoinType.JOIN_TYPE_LEFT_ANTI + JoinRel.JoinType.JOIN_TYPE_LEFT_SEMI case _ => // TODO: Support cross join with Cross Rel JoinRel.JoinType.UNRECOGNIZED diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala index c2648f29ec4c..793d733abf96 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/execution/CHWindowGroupLimitExecTransformer.scala @@ -182,6 +182,6 @@ case class CHWindowGroupLimitExecTransformer( val currRel = getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Group Limit Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala new file mode 100644 index 000000000000..ad2b22ba6cc6 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ConvertWindowToAggregate.scala @@ -0,0 +1,138 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.execution._ +import org.apache.gluten.expression.WindowFunctionsBuilder + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.rules.Rule +// import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types._ + +// When to find the first rows of partitions by window function, we can convert it to aggregate +// function. +case class ConverRowNumbertWindowToAggregateRule(spark: SparkSession) + extends Rule[SparkPlan] + with Logging { + + override def apply(plan: SparkPlan): SparkPlan = { + if (!CHBackendSettings.enableConvertWindowGroupLimitToAggregate) { + return plan + } + plan.transformUp { + case filter @ FilterExecTransformer( + condition, + window @ WindowExecTransformer( + windowExpressions, + partitionSpec, + orderSpec, + sort @ SortExecTransformer(_, _, _, _))) => + if ( + !isSupportedWindowFunction(windowExpressions) || !isTopKLimitFilter( + condition, + windowExpressions(0)) + ) { + logDebug( + s"xxx Not Supported case for converting window to aggregate. is topk limit: " + + s"${isTopKLimitFilter(condition, windowExpressions(0))}. is supported window " + + s"function: ${isSupportedWindowFunction(windowExpressions)}") + filter + } else { + val limit = getLimit(condition.asInstanceOf[BinaryComparison]) + if (limit < 1 || limit > 100) { + filter + } else { + val groupLimit = CHAggregateGroupLimitExecTransformer( + partitionSpec, + orderSpec, + extractWindowFunction(windowExpressions(0)), + sort.child.output ++ Seq(windowExpressions(0).toAttribute), + limit, + sort.child + ) + groupLimit + } + } + } + } + + def getLimit(e: BinaryComparison): Int = { + e match { + case _: EqualTo => evalIntLiteral(e.right).get + case _: LessThanOrEqual => evalIntLiteral(e.right).get + case _: LessThan => evalIntLiteral(e.right).get - 1 + case _ => throw new GlutenException(s"Unsupported comparison: $e") + } + } + + def evalIntLiteral(expr: Expression): Option[Int] = { + expr match { + case int @ Literal(value, IntegerType) => Some(value.asInstanceOf[Int]) + case long @ Literal(value, LongType) => Some(value.asInstanceOf[Long].toInt) + case _ => None + } + } + + def extractWindowFunction(windowExpression: NamedExpression): Expression = { + val aliasExpr = windowExpression.asInstanceOf[Alias] + WindowFunctionsBuilder.extractWindowExpression(aliasExpr.child).windowFunction + } + + def isSupportedWindowFunction(windowExpressions: Seq[NamedExpression]): Boolean = { + if (windowExpressions.length != 1) { + return false + } + val windowFunction = extractWindowFunction(windowExpressions(0)) + windowFunction match { + case _: RowNumber => true + case _ => false + } + } + + // If the filter condition is a constant limit condition, return the limit value. + def isTopKLimitFilter(condition: Expression, windowExpression: NamedExpression): Boolean = { + def isWindowFunctionResult( + condition: Expression, + windowExpression: NamedExpression): Boolean = { + condition match { + case attr: Attribute => + attr.semanticEquals(windowExpression.toAttribute) + case _ => false + } + } + if (!condition.isInstanceOf[BinaryComparison]) { + return false + } + val binaryComparison = condition.asInstanceOf[BinaryComparison] + val constLimit = evalIntLiteral(binaryComparison.right) + if (!isWindowFunctionResult(binaryComparison.left, windowExpression) || !constLimit.isDefined) { + return false + } + binaryComparison match { + case _: EqualTo => constLimit.get == 1 + case _: LessThanOrEqual | _: LessThan => true + case _ => false + } + } + +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala new file mode 100644 index 000000000000..f2a0a549bc7e --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/ExtendedColumnPruning.scala @@ -0,0 +1,366 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction +import org.apache.spark.sql.catalyst.optimizer.GeneratorNestedColumnAliasing.canPruneGenerator +import org.apache.spark.sql.catalyst.optimizer.NestedColumnAliasing +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.AlwaysProcess +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +import scala.collection.mutable + +object ExtendedGeneratorNestedColumnAliasing { + def unapply(plan: LogicalPlan): Option[LogicalPlan] = plan match { + case pj @ Project(projectList, f @ Filter(condition, g: Generate)) + if canPruneGenerator(g.generator) && + GlutenConfig.getConf.enableExtendedColumnPruning && + (SQLConf.get.nestedPruningOnExpressions || SQLConf.get.nestedSchemaPruningEnabled) => + val attrToExtractValues = + getAttributeToExtractValues(projectList ++ g.generator.children :+ condition, Seq.empty) + if (attrToExtractValues.isEmpty) { + return None + } + + val generatorOutputSet = AttributeSet(g.qualifiedGeneratorOutput) + var (attrToExtractValuesOnGenerator, attrToExtractValuesNotOnGenerator) = + attrToExtractValues.partition { + case (attr, _) => + attr.references.subsetOf(generatorOutputSet) + } + + val pushedThrough = rewritePlanWithAliases(pj, attrToExtractValuesNotOnGenerator) + + // We cannot push through if the child of generator is `MapType`. + g.generator.children.head.dataType match { + case _: MapType => return Some(pushedThrough) + case _ => + } + + if (!g.generator.isInstanceOf[ExplodeBase]) { + return Some(pushedThrough) + } + + // In spark3.2, we could not reuse [[NestedColumnAliasing.getAttributeToExtractValues]] + // which only accepts 2 arguments. Instead we redefine it in current file to avoid moving + // this rule to gluten-shims + attrToExtractValuesOnGenerator = getAttributeToExtractValues( + attrToExtractValuesOnGenerator.flatMap(_._2).toSeq, + Seq.empty, + collectNestedGetStructFields) + + val nestedFieldsOnGenerator = attrToExtractValuesOnGenerator.values.flatten.toSet + if (nestedFieldsOnGenerator.isEmpty) { + return Some(pushedThrough) + } + + // Multiple or single nested column accessors. + // E.g. df.select(explode($"items").as("item")).select($"item.a", $"item.b") + pushedThrough match { + case p2 @ Project(_, f2 @ Filter(_, g2: Generate)) => + val nestedFieldsOnGeneratorSeq = nestedFieldsOnGenerator.toSeq + val nestedFieldToOrdinal = nestedFieldsOnGeneratorSeq.zipWithIndex.toMap + val rewrittenG = g2.transformExpressions { + case e: ExplodeBase => + val extractors = nestedFieldsOnGeneratorSeq.map(replaceGenerator(e, _)) + val names = extractors.map { + case g: GetStructField => Literal(g.extractFieldName) + case ga: GetArrayStructFields => Literal(ga.field.name) + case other => + throw new IllegalStateException( + s"Unreasonable extractor " + + "after replaceGenerator: $other") + } + val zippedArray = ArraysZip(extractors, names) + e.withNewChildren(Seq(zippedArray)) + } + // As we change the child of the generator, its output data type must be updated. + val updatedGeneratorOutput = rewrittenG.generatorOutput + .zip( + rewrittenG.generator.elementSchema.map( + f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)())) + .map { + case (oldAttr, newAttr) => + newAttr.withExprId(oldAttr.exprId).withName(oldAttr.name) + } + assert( + updatedGeneratorOutput.length == rewrittenG.generatorOutput.length, + "Updated generator output must have the same length " + + "with original generator output." + ) + val updatedGenerate = rewrittenG.copy(generatorOutput = updatedGeneratorOutput) + + // Replace nested column accessor with generator output. + val attrExprIdsOnGenerator = attrToExtractValuesOnGenerator.keys.map(_.exprId).toSet + val updatedFilter = f2.withNewChildren(Seq(updatedGenerate)).transformExpressions { + case f: GetStructField if nestedFieldsOnGenerator.contains(f) => + replaceGetStructField( + f, + updatedGenerate.output, + attrExprIdsOnGenerator, + nestedFieldToOrdinal) + } + + val updatedProject = p2.withNewChildren(Seq(updatedFilter)).transformExpressions { + case f: GetStructField if nestedFieldsOnGenerator.contains(f) => + replaceGetStructField( + f, + updatedFilter.output, + attrExprIdsOnGenerator, + nestedFieldToOrdinal) + } + + Some(updatedProject) + case other => + throw new IllegalStateException(s"Unreasonable plan after optimization: $other") + } + case _ => + None + } + + /** + * Returns two types of expressions: + * - Root references that are individually accessed + * - [[GetStructField]] or [[GetArrayStructFields]] on top of other [[ExtractValue]]s or special + * expressions. + */ + private def collectRootReferenceAndExtractValue(e: Expression): Seq[Expression] = e match { + case _: AttributeReference => Seq(e) + case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => Seq(e) + case GetArrayStructFields( + _: MapValues | _: MapKeys | _: ExtractValue | _: AttributeReference, + _, + _, + _, + _) => + Seq(e) + case es if es.children.nonEmpty => es.children.flatMap(collectRootReferenceAndExtractValue) + case _ => Seq.empty + } + + /** + * Creates a map from root [[Attribute]]s to non-redundant nested [[ExtractValue]]s. Nested field + * accessors of `exclusiveAttrs` are not considered in nested fields aliasing. + */ + private def getAttributeToExtractValues( + exprList: Seq[Expression], + exclusiveAttrs: Seq[Attribute], + extractor: (Expression) => Seq[Expression] = collectRootReferenceAndExtractValue) + : Map[Attribute, Seq[ExtractValue]] = { + + val nestedFieldReferences = new mutable.ArrayBuffer[ExtractValue]() + val otherRootReferences = new mutable.ArrayBuffer[AttributeReference]() + exprList.foreach { + e => + extractor(e).foreach { + // we can not alias the attr from lambda variable whose expr id is not available + case ev: ExtractValue if ev.find(_.isInstanceOf[NamedLambdaVariable]).isEmpty => + if (ev.references.size == 1) { + nestedFieldReferences.append(ev) + } + case ar: AttributeReference => otherRootReferences.append(ar) + case _ => // ignore + } + } + val exclusiveAttrSet = AttributeSet(exclusiveAttrs ++ otherRootReferences) + + // Remove cosmetic variations when we group extractors by their references + nestedFieldReferences + .filter(!_.references.subsetOf(exclusiveAttrSet)) + .groupBy(_.references.head.canonicalized.asInstanceOf[Attribute]) + .flatMap { + case (attr: Attribute, nestedFields: collection.Seq[ExtractValue]) => + // Check if `ExtractValue` expressions contain any aggregate functions in their tree. + // Those that do should not have an alias generated as it can lead to pushing the + // aggregate down into a projection. + def containsAggregateFunction(ev: ExtractValue): Boolean = + ev.find(_.isInstanceOf[AggregateFunction]).isDefined + + // Remove redundant [[ExtractValue]]s if they share the same parent nest field. + // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`. + // Because `a.b` requires all of the inner fields of `b`, we cannot prune `a.b.c`. + val dedupNestedFields = nestedFields + .filter { + // See [[collectExtractValue]]: we only need to deal with [[GetArrayStructFields]] and + // [[GetStructField]] + case e @ (_: GetStructField | _: GetArrayStructFields) => + val child = e.children.head + nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty) + case _ => true + } + .distinct + // Discard [[ExtractValue]]s that contain aggregate functions. + .filterNot(containsAggregateFunction) + + // If all nested fields of `attr` are used, we don't need to introduce new aliases. + // By default, the [[ColumnPruning]] rule uses `attr` already. + // Note that we need to remove cosmetic variations first, so we only count a + // nested field once. + val numUsedNestedFields = dedupNestedFields + .map(_.canonicalized) + .distinct + .map(nestedField => totalFieldNum(nestedField.dataType)) + .sum + if (dedupNestedFields.nonEmpty && numUsedNestedFields < totalFieldNum(attr.dataType)) { + Some((attr, dedupNestedFields.toSeq)) + } else { + None + } + } + } + + /** + * Return total number of fields of this type. This is used as a threshold to use nested column + * pruning. It's okay to underestimate. If the number of reference is bigger than this, the parent + * reference is used instead of nested field references. + */ + private def totalFieldNum(dataType: DataType): Int = dataType match { + case StructType(fields) => fields.map(f => totalFieldNum(f.dataType)).sum + case ArrayType(elementType, _) => totalFieldNum(elementType) + case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) + case _ => 1 // UDT and others + } + + private def replaceGetStructField( + g: GetStructField, + input: Seq[Attribute], + attrExprIdsOnGenerator: Set[ExprId], + nestedFieldToOrdinal: Map[ExtractValue, Int]): Expression = { + val attr = input.find(a => attrExprIdsOnGenerator.contains(a.exprId)) + attr match { + case Some(a) => + val ordinal = nestedFieldToOrdinal(g) + GetStructField(a, ordinal, g.name) + case None => g + } + } + + /** Replace the reference attribute of extractor expression with generator input. */ + private def replaceGenerator(generator: ExplodeBase, expr: Expression): Expression = { + expr match { + case a: Attribute if expr.references.contains(a) => + generator.child + case g: GetStructField => + // We cannot simply do a transformUp instead because if we replace the attribute + // `extractFieldName` could cause `ClassCastException` error. We need to get the + // field name before replacing down the attribute/other extractor. + val fieldName = g.extractFieldName + val newChild = replaceGenerator(generator, g.child) + ExtractValue(newChild, Literal(fieldName), SQLConf.get.resolver) + case other => + other.mapChildren(replaceGenerator(generator, _)) + } + } + + // This function collects all GetStructField*(attribute) from the passed in expression. + // GetStructField* means arbitrary levels of nesting. + private def collectNestedGetStructFields(e: Expression): Seq[Expression] = { + // The helper function returns a tuple of + // (nested GetStructField including the current level, all other nested GetStructField) + def helper(e: Expression): (Seq[Expression], Seq[Expression]) = e match { + case _: AttributeReference => (Seq(e), Seq.empty) + case gsf: GetStructField => + val child_res = helper(gsf.child) + (child_res._1.map(p => gsf.withNewChildren(Seq(p))), child_res._2) + case other => + val child_res = other.children.map(helper) + val child_res_combined = (child_res.flatMap(_._1), child_res.flatMap(_._2)) + (Seq.empty, child_res_combined._1 ++ child_res_combined._2) + } + + val res = helper(e) + (res._1 ++ res._2).filterNot(_.isInstanceOf[Attribute]) + } + + private def rewritePlanWithAliases( + plan: LogicalPlan, + attributeToExtractValues: Map[Attribute, Seq[ExtractValue]]): LogicalPlan = { + val attributeToExtractValuesAndAliases = + attributeToExtractValues.map { + case (attr, evSeq) => + val evAliasSeq = evSeq.map { + ev => + val fieldName = ev match { + case g: GetStructField => g.extractFieldName + case g: GetArrayStructFields => g.field.name + } + ev -> Alias(ev, s"_extract_$fieldName")() + } + + attr -> evAliasSeq + } + + val nestedFieldToAlias = attributeToExtractValuesAndAliases.values.flatten.map { + case (field, alias) => field.canonicalized -> alias + }.toMap + + // A reference attribute can have multiple aliases for nested fields. + val attrToAliases = + AttributeMap(attributeToExtractValuesAndAliases.mapValues(_.map(_._2)).toSeq) + + plan match { + // Project(Filter(Generate)) + case Project(projectList, f @ Filter(condition, g: Generate)) => + val newProjectList = NestedColumnAliasing.getNewProjectList(projectList, nestedFieldToAlias) + val newCondition = getNewExpression(condition, nestedFieldToAlias) + val newGenerator = getNewExpression(g.generator, nestedFieldToAlias).asInstanceOf[Generator] + + val tmpG = NestedColumnAliasing + .replaceWithAliases(g, nestedFieldToAlias, attrToAliases) + .asInstanceOf[Generate] + val newG = Generate( + newGenerator, + tmpG.unrequiredChildIndex, + tmpG.outer, + tmpG.qualifier, + tmpG.generatorOutput, + tmpG.children.head) + val newF = Filter(newCondition, newG) + val newP = Project(newProjectList, newF) + newP + case _ => plan + } + } + + private def getNewExpression( + expr: Expression, + nestedFieldToAlias: Map[Expression, Alias]): Expression = { + expr.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f.canonicalized) => + nestedFieldToAlias(f.canonicalized).toAttribute + } + } +} + +// ExtendedColumnPruning process Project(Filter(Generate)), +// which is ignored by vanilla spark in optimization rule: ColumnPruning +class ExtendedColumnPruning(spark: SparkSession) extends Rule[LogicalPlan] with Logging { + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(AlwaysProcess.fn) { + case ExtendedGeneratorNestedColumnAliasing(rewrittenPlan) => rewrittenPlan + case p => p + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala index ec465a3c1506..6a788617a6c4 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/FallbackBroadcastHashJoinRules.scala @@ -18,8 +18,7 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} @@ -38,7 +37,7 @@ import scala.util.control.Breaks.{break, breakable} // queryStagePrepRules. case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { - val columnarConf: GlutenConfig = GlutenConfig.getConf + val glutenConf: GlutenConfig = GlutenConfig.getConf plan.foreach { case bhj: BroadcastHashJoinExec => val buildSidePlan = bhj.buildSide match { @@ -53,8 +52,8 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend case Some(exchange @ BroadcastExchangeExec(mode, child)) => val isTransformable = if ( - !columnarConf.enableColumnarBroadcastExchange || - !columnarConf.enableColumnarBroadcastJoin + !glutenConf.enableColumnarBroadcastExchange || + !glutenConf.enableColumnarBroadcastJoin ) { ValidationResult.failed( "columnar broadcast exchange is disabled or " + @@ -117,19 +116,19 @@ case class FallbackBroadcastHashJoinPrepQueryStage(session: SparkSession) extend if (FallbackTags.nonEmpty(bnlj)) { ValidationResult.failed("broadcast join is already tagged as not transformable") } else { - val transformer = BackendsApiManager.getSparkPlanExecApiInstance + val bnljTransformer = BackendsApiManager.getSparkPlanExecApiInstance .genBroadcastNestedLoopJoinExecTransformer( bnlj.left, bnlj.right, bnlj.buildSide, bnlj.joinType, bnlj.condition) - val isTransformable = transformer.doValidate() - if (isTransformable.ok()) { + val isBnljTransformable = bnljTransformer.doValidate() + if (isBnljTransformable.ok()) { val exchangeTransformer = ColumnarBroadcastExchangeExec(mode, child) exchangeTransformer.doValidate() } else { - isTransformable + isBnljTransformable } } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala new file mode 100644 index 000000000000..0f7752339707 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/LazyAggregateExpandRule.scala @@ -0,0 +1,388 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.backendsapi.clickhouse.CHBackendSettings +import org.apache.gluten.execution._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types._ + +/* + * For aggregation with grouping sets, we need to expand the grouping sets + * to individual group by. + * 1. It need to make copies of the original data. + * 2. run the aggregation on the multi copied data. + * Both of these two are expensive. + * + * We could do this as following + * 1. Run the aggregation on full grouping keys. + * 2. Expand the aggregation result to the full grouping sets. + * 3. Run the aggregation on the expanded data. + * + * So the plan is transformed from + * expand -> partial aggregating -> shuffle -> final merge aggregating + * to + * partial aggregating -> expand -> shuffle -> final merge aggregating + * + * Notice: + * If the aggregation involves distinct, we can't do this optimization. + */ + +case class LazyAggregateExpandRule(session: SparkSession) extends Rule[SparkPlan] with Logging { + override def apply(plan: SparkPlan): SparkPlan = { + logDebug(s"xxx enable lazy aggregate expand: ${CHBackendSettings.enableLazyAggregateExpand}") + if (!CHBackendSettings.enableLazyAggregateExpand) { + return plan + } + plan.transformUp { + case shuffle @ ColumnarShuffleExchangeExec( + HashPartitioning(hashExpressions, _), + CHHashAggregateExecTransformer( + _, + groupingExpressions, + aggregateExpressions, + _, + _, + resultExpressions, + ExpandExecTransformer(projections, output, child)), + _, + _, + _ + ) => + logDebug(s"xxx match plan:$shuffle") + val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer] + val expand = partialAggregate.child.asInstanceOf[ExpandExecTransformer] + logDebug( + s"xxx partialAggregate: groupingExpressions:" + + s"${partialAggregate.groupingExpressions}\n" + + s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" + + s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + if (doValidation(partialAggregate, expand, shuffle)) { + + val attributesToReplace = buildReplaceAttributeMap(expand) + logDebug(s"xxx attributesToReplace: $attributesToReplace") + + val newPartialAggregate = buildAheadAggregateExec( + partialAggregate, + expand, + attributesToReplace + ) + + val newExpand = buildPostExpandExec( + expand, + partialAggregate, + newPartialAggregate, + attributesToReplace + ) + + val newShuffle = shuffle.copy(child = newExpand) + logDebug(s"xxx new plan: $newShuffle") + newShuffle + } else { + shuffle + } + case shuffle @ ColumnarShuffleExchangeExec( + HashPartitioning(hashExpressions, _), + CHHashAggregateExecTransformer( + _, + groupingExpressions, + aggregateExpressions, + _, + _, + resultExpressions, + FilterExecTransformer(_, ExpandExecTransformer(projections, output, child))), + _, + _, + _ + ) => + val partialAggregate = shuffle.child.asInstanceOf[CHHashAggregateExecTransformer] + val filter = partialAggregate.child.asInstanceOf[FilterExecTransformer] + val expand = filter.child.asInstanceOf[ExpandExecTransformer] + logDebug( + s"xxx partialAggregate: groupingExpressions:" + + s"${partialAggregate.groupingExpressions}\n" + + s"aggregateAttributes:${partialAggregate.aggregateAttributes}\n" + + s"aggregateExpressions:${partialAggregate.aggregateExpressions}\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + if (doValidation(partialAggregate, expand, shuffle)) { + val attributesToReplace = buildReplaceAttributeMap(expand) + logDebug(s"xxx attributesToReplace: $attributesToReplace") + + val newPartialAggregate = buildAheadAggregateExec( + partialAggregate, + expand, + attributesToReplace + ) + + val newExpand = buildPostExpandExec( + expand, + partialAggregate, + newPartialAggregate, + attributesToReplace + ) + + val newFilter = filter.copy(child = newExpand) + + val newShuffle = shuffle.copy(child = newFilter) + logDebug(s"xxx new plan: $newShuffle") + newShuffle + + } else { + shuffle + } + } + } + + // Just enable for simple cases. Some of cases that are not supported: + // 1. select count(a),count(b), count(1), count(distinct(a)), count(distinct(b)) from values + // (1, null), (2,2) as data(a,b); + // 2. select n_name, count(distinct n_regionkey) as col1, + // count(distinct concat(n_regionkey, n_nationkey)) as col2 from + // nation group by n_name; + def doValidation( + aggregate: CHHashAggregateExecTransformer, + expand: ExpandExecTransformer, + shuffle: ColumnarShuffleExchangeExec): Boolean = { + // all grouping keys must be attribute references + val expandOutputAttributes = expand.child.output.toSet + if ( + !aggregate.groupingExpressions.forall( + e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal]) + ) { + logDebug(s"xxx Not all grouping expression are attribute references") + return false + } + // all shuffle keys must be attribute references + if ( + !shuffle.outputPartitioning + .asInstanceOf[HashPartitioning] + .expressions + .forall(e => e.isInstanceOf[Attribute] || e.isInstanceOf[Literal]) + ) { + logDebug(s"xxx Not all shuffle hash expression are attribute references") + return false + } + + // 1. for safty, we don't enbale this optimization for all aggregate functions. + // 2. if any aggregate function uses attributes which is not from expand's child, we don't + // enable this + if ( + !aggregate.aggregateExpressions.forall { + e => + isValidAggregateFunction(e) && + e.aggregateFunction.references.forall( + attr => expandOutputAttributes.find(_.semanticEquals(attr)).isDefined) + } + ) { + logDebug(s"xxx Some aggregate functions are not supported") + return false + } + + // get the group id's position in the expand's output + val gidIndex = findGroupingIdIndex(expand) + gidIndex != -1 + } + + // group id column doesn't have a fixed position, so we need to find it. + def findGroupingIdIndex(expand: ExpandExecTransformer): Int = { + def isValidGroupIdColumn(e: Expression, gids: Set[Long]): Long = { + if (!e.isInstanceOf[Literal]) { + return -1 + } + val literalValue = e.asInstanceOf[Literal].value + e.dataType match { + case _: LongType => + if (gids.contains(literalValue.asInstanceOf[Long])) { + -1 + } else { + literalValue.asInstanceOf[Long] + } + case _: IntegerType => + if (gids.contains(literalValue.asInstanceOf[Int].toLong)) { + -1 + } else { + literalValue.asInstanceOf[Int].toLong + } + case _ => -1 + } + } + + var groupIdIndexes = Seq[Int]() + for (col <- 0 until expand.output.length) { + val expandCol = expand.projections(0)(col) + // gids should be unique + var gids = Set[Long]() + if (isValidGroupIdColumn(expandCol, gids) != -1) { + if ( + expand.projections.forall { + projection => + val res = isValidGroupIdColumn(projection(col), gids) + gids += res + res != -1 + } + ) { + groupIdIndexes +:= col + } + } + } + if (groupIdIndexes.length == 1) { + logDebug(s"xxx gid is at pos ${groupIdIndexes(0)}") + groupIdIndexes(0) + } else { + -1 + } + } + + // Some of aggregate functions' output columns are not consistent with the output of gluten. + // - average: in partial aggregation, the outputs are sum and count, but gluten only generates one + // column, avg. + // - sum: if the input's type is decimal, the output are sum and isEmpty, but gluten doesn't use + // the isEmpty column. + def isValidAggregateFunction(aggregateExpression: AggregateExpression): Boolean = { + if (aggregateExpression.filter.isDefined) { + return false + } + aggregateExpression.aggregateFunction match { + case _: Count => true + case _: Max => true + case _: Min => true + case _: Average => true + case _: Sum => true + case _ => false + } + } + + def getReplaceAttribute( + toReplace: Attribute, + attributesToReplace: Map[Attribute, Attribute]): Attribute = { + val kv = attributesToReplace.find(kv => kv._1.semanticEquals(toReplace)) + kv match { + case Some((_, v)) => v + case None => toReplace + } + } + + def buildReplaceAttributeMap(expand: ExpandExecTransformer): Map[Attribute, Attribute] = { + var fullExpandProjection = Seq[Expression]() + for (i <- 0 until expand.projections(0).length) { + val attr = expand.projections.find(x => x(i).isInstanceOf[Attribute]) match { + case Some(projection) => projection(i).asInstanceOf[Attribute] + case None => null + } + fullExpandProjection = fullExpandProjection :+ attr + } + + var attributeMap = Map[Attribute, Attribute]() + for (i <- 0 until expand.output.length) { + if (fullExpandProjection(i).isInstanceOf[Attribute]) { + attributeMap += (expand.output(i) -> fullExpandProjection(i).asInstanceOf[Attribute]) + } + } + attributeMap + } + + def buildPostExpandProjections( + originalExpandProjections: Seq[Seq[Expression]], + originalExpandOutput: Seq[Attribute], + newExpandOutput: Seq[Attribute]): Seq[Seq[Expression]] = { + val newExpandProjections = originalExpandProjections.map { + projection => + newExpandOutput.map { + attr => + val index = originalExpandOutput.indexWhere(_.semanticEquals(attr)) + if (index != -1) { + projection(index) + } else { + attr + } + } + } + newExpandProjections + } + + // 1. make expand's child be aggregate's child + // 2. replace the attributes in groupingExpressions and resultExpressions as needed + def buildAheadAggregateExec( + partialAggregate: CHHashAggregateExecTransformer, + expand: ExpandExecTransformer, + attributesToReplace: Map[Attribute, Attribute]): SparkPlan = { + val groupIdAttribute = expand.output(findGroupingIdIndex(expand)) + + // New grouping expressions should include the group id column + val groupingExpressions = + partialAggregate.groupingExpressions + .filter( + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + .distinct + logDebug( + s"xxx newGroupingExpresion: $groupingExpressions,\n" + + s"groupingExpressions: ${partialAggregate.groupingExpressions}") + + // Remove group id column from result expressions + val groupingKeysCount = + partialAggregate.resultExpressions.length - partialAggregate.aggregateExpressions.length + var resultExpressions = partialAggregate.resultExpressions + .slice(0, groupingKeysCount) + .filter( + e => + !e.toAttribute.semanticEquals(groupIdAttribute) && + attributesToReplace.find(kv => kv._1.semanticEquals(e.toAttribute)).isDefined) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + resultExpressions = resultExpressions ++ partialAggregate.resultExpressions + .slice(groupingKeysCount, partialAggregate.resultExpressions.length) + .map(e => getReplaceAttribute(e.toAttribute, attributesToReplace)) + logDebug( + s"xxx newResultExpressions: $resultExpressions\n" + + s"resultExpressions:${partialAggregate.resultExpressions}") + partialAggregate.copy( + groupingExpressions = groupingExpressions, + resultExpressions = resultExpressions.distinct, + child = expand.child) + } + + def buildPostExpandExec( + expand: ExpandExecTransformer, + partialAggregate: CHHashAggregateExecTransformer, + child: SparkPlan, + attributesToReplace: Map[Attribute, Attribute]): SparkPlan = { + // The output of the native plan is not completely consistent with Spark. + val aggregateOutput = partialAggregate.output + logDebug(s"xxx aggregateResultAttributes: ${partialAggregate.aggregateResultAttributes}") + logDebug(s"xxx aggregateOutput: $aggregateOutput") + + val expandProjections = buildPostExpandProjections( + expand.projections, + expand.output, + aggregateOutput + ) + logDebug(s"xxx expandProjections: $expandProjections\nprojections:${expand.projections}") + ExpandExecTransformer(expandProjections, aggregateOutput, child) + } + +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala index a3fab3c954ee..9335fd8829a9 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/PushdownAggregatePreProjectionAheadExpand.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan // If there is an expression (not a attribute) in an aggregation function's -// parameters. It will introduce a pr-projection to calculate the expression +// parameters. It will introduce a pre-projection to calculate the expression // at first, and make all the parameters be attributes. // If it's a aggregation with grouping set, this pre-projection is placed after // expand operator. This is not efficient, we cannot move this pre-projection @@ -83,7 +83,7 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession) val originInputAttributes = aheadProjectExprs.filter(e => isAttributeOrLiteral(e)) val preProjectExprs = aheadProjectExprs.filter(e => !isAttributeOrLiteral(e)) - if (preProjectExprs.length == 0) { + if (preProjectExprs.isEmpty) { return hashAggregate } @@ -93,11 +93,31 @@ case class PushdownAggregatePreProjectionAheadExpand(session: SparkSession) return hashAggregate } + def projectInputExists(expr: Expression, inputs: Seq[Attribute]): Boolean = { + expr.children.foreach { + case a: Attribute => + return inputs.exists(i => i.name.equals(a.name) && i.exprId.equals(a.exprId)) + case p: Expression => + return projectInputExists(p, inputs) + case _ => + return true + } + true + } + + val couldPushDown = preProjectExprs.forall { + case p: Expression => projectInputExists(p, rootChild.output) + case _ => true + } + + if (!couldPushDown) { + return hashAggregate; + } + // The new ahead project node will take rootChild's output and preProjectExprs as the // the projection expressions. val aheadProject = ProjectExecTransformer(rootChild.output ++ preProjectExprs, rootChild) val aheadProjectOuput = aheadProject.output - val preProjectOutputAttrs = aheadProjectOuput.filter( e => !originInputAttributes.exists(_.exprId.equals(e.asInstanceOf[NamedExpression].exprId))) diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala new file mode 100644 index 000000000000..7f378b5a41a0 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/RemoveDuplicatedColumns.scala @@ -0,0 +1,126 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.execution._ + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.CHColumnarToRowExec + +/* + * CH doesn't support will for duplicate columns in the a block. + * Most of the cases that introduce duplicate columns are from group by. + */ +case class RemoveDuplicatedColumns(session: SparkSession) extends Rule[SparkPlan] with Logging { + override def apply(plan: SparkPlan): SparkPlan = { + visitPlan(plan) + } + + private def visitPlan(plan: SparkPlan): SparkPlan = { + plan match { + case c2r @ CHColumnarToRowExec(hashAgg: CHHashAggregateExecTransformer) => + // This is a special case. Use the result from aggregation as the input of sink. + // We need to make the schema same as the input of sink. + val newChildren = hashAgg.children.map(visitPlan) + val newHashAgg = uniqueHashAggregateColumns(hashAgg) + .withNewChildren(newChildren) + .asInstanceOf[CHHashAggregateExecTransformer] + if (newHashAgg.resultExpressions.length != hashAgg.resultExpressions.length) { + val project = ProjectExecTransformer(hashAgg.resultExpressions, newHashAgg) + c2r.copy(child = project) + } else { + c2r.copy(child = newHashAgg) + } + case hashAgg: CHHashAggregateExecTransformer => + val newChildren = hashAgg.children.map(visitPlan) + val newHashAgg = uniqueHashAggregateColumns(hashAgg) + newHashAgg.withNewChildren(newChildren) + case _ => + plan.withNewChildren(plan.children.map(visitPlan)) + } + } + + private def unwrapAliasNamedExpression(e: NamedExpression): NamedExpression = { + e match { + case a: Alias => + if (a.child.isInstanceOf[NamedExpression]) { + a.child.asInstanceOf[NamedExpression] + } else { + a + } + case _ => e + } + } + private def unwrapAliasExpression(e: Expression): Expression = { + e match { + case a: Alias => + if (a.child.isInstanceOf[Expression]) { + a.child.asInstanceOf[Expression] + } else { + a + } + case _ => e + } + } + + private def uniqueNamedExpressions( + groupingExpressions: Seq[NamedExpression]): Seq[NamedExpression] = { + var addedExpression = Seq[NamedExpression]() + groupingExpressions.foreach { + e => + val unwrapped = unwrapAliasNamedExpression(e) + if ( + !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal] + ) { + addedExpression = addedExpression :+ unwrapped + } + } + addedExpression + } + + private def uniqueExpressions(expressions: Seq[Expression]): Seq[Expression] = { + var addedExpression = Seq[Expression]() + expressions.foreach { + e => + val unwrapped = unwrapAliasExpression(e) + if ( + !addedExpression.exists(_.semanticEquals(unwrapped)) && !unwrapped.isInstanceOf[Literal] + ) { + addedExpression = addedExpression :+ unwrapped + } + } + addedExpression + } + + private def uniqueHashAggregateColumns( + hashAgg: CHHashAggregateExecTransformer): CHHashAggregateExecTransformer = { + val newGroupingExpressions = uniqueNamedExpressions(hashAgg.groupingExpressions) + val newResultExpressions = uniqueNamedExpressions(hashAgg.resultExpressions) + if (newResultExpressions.length != hashAgg.resultExpressions.length) { + hashAgg + .copy( + groupingExpressions = newGroupingExpressions, + resultExpressions = newResultExpressions) + } else { + hashAgg + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala new file mode 100644 index 000000000000..8ab78dcff9bc --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/extension/WriteFilesWithBucketValue.scala @@ -0,0 +1,76 @@ +/* + * 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.gluten.extension + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.sql.catalyst.expressions.{Alias, BitwiseAnd, Expression, HiveHash, Literal, Pmod} +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.datasources.WriteFilesExec + +/** + * Wrap with bucket value to specify the bucket file name in native write. Native writer will remove + * this value in the final output. + */ +object WriteFilesWithBucketValue extends Rule[SparkPlan] { + + val optionForHiveCompatibleBucketWrite = "__hive_compatible_bucketed_table_insertion__" + + override def apply(plan: SparkPlan): SparkPlan = { + if ( + GlutenConfig.getConf.enableGluten + && GlutenConfig.getConf.enableNativeWriter.getOrElse(false) + ) { + plan.transformDown { + case writeFiles: WriteFilesExec if writeFiles.bucketSpec.isDefined => + val bucketIdExp = getWriterBucketIdExp(writeFiles) + val wrapBucketValue = ProjectExec( + writeFiles.child.output :+ Alias(bucketIdExp, "__bucket_value__")(), + writeFiles.child) + writeFiles.copy(child = wrapBucketValue) + } + } else { + plan + } + } + + private def getWriterBucketIdExp(writeFilesExec: WriteFilesExec): Expression = { + val partitionColumns = writeFilesExec.partitionColumns + val outputColumns = writeFilesExec.child.output + val dataColumns = outputColumns.filterNot(partitionColumns.contains) + val bucketSpec = writeFilesExec.bucketSpec.get + val bucketColumns = bucketSpec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get) + if (writeFilesExec.options.getOrElse(optionForHiveCompatibleBucketWrite, "false") == "true") { + val hashId = BitwiseAnd(HiveHash(bucketColumns), Literal(Int.MaxValue)) + Pmod(hashId, Literal(bucketSpec.numBuckets)) + // The bucket file name prefix is following Hive, Presto and Trino conversion, so this + // makes sure Hive bucketed table written by Spark, can be read by other SQL engines. + // + // Hive: `org.apache.hadoop.hive.ql.exec.Utilities#getBucketIdFromFile()`. + // Trino: `io.trino.plugin.hive.BackgroundHiveSplitLoader#BUCKET_PATTERNS`. + + } else { + // Spark bucketed table: use `HashPartitioning.partitionIdExpression` as bucket id + // expression, so that we can guarantee the data distribution is same between shuffle and + // bucketed data source, which enables us to only shuffle one side when join a bucketed + // table and a normal one. + HashPartitioning(bucketColumns, bucketSpec.numBuckets).partitionIdExpression + } + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala index e1e0f7c11a09..7d81467e978f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala @@ -38,6 +38,9 @@ object MetricsUtil extends Logging { j.metricsUpdater(), // must put the buildPlan first Seq(treeifyMetricsUpdaters(j.buildPlan), treeifyMetricsUpdaters(j.streamedPlan))) + case t: TransformSupport if t.metricsUpdater() == MetricsUpdater.None => + assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator") + treeifyMetricsUpdaters(t.children.head) case t: TransformSupport => MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) case _ => diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala index 9dcb7ee3c41d..d6511f7a4a29 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/utils/PlanNodesUtil.scala @@ -22,13 +22,17 @@ import org.apache.gluten.substrait.expression.ExpressionNode import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} import org.apache.gluten.substrait.rel.RelBuilder -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, Expression} import com.google.common.collect.Lists +import java.util + +import scala.collection.JavaConverters._ + object PlanNodesUtil { - def genProjectionsPlanNode(key: Expression, output: Seq[Attribute]): PlanNode = { + def genProjectionsPlanNode(key: Seq[Expression], output: Seq[Attribute]): PlanNode = { val context = new SubstraitContext var operatorId = context.nextOperatorId("ClickHouseBuildSideRelationReadIter") @@ -36,41 +40,36 @@ object PlanNodesUtil { val nameList = ConverterUtils.collectAttributeNamesWithExprId(output) val readRel = RelBuilder.makeReadRelForInputIterator(typeList, nameList, context, operatorId) - // replace attribute to BoundRefernce according to the output - val newBoundRefKey = key.transformDown { - case expression: AttributeReference => - val columnInOutput = output.zipWithIndex.filter { - p: (Attribute, Int) => p._1.exprId == expression.exprId || p._1.name == expression.name - } - if (columnInOutput.isEmpty) { - throw new IllegalStateException( - s"Key $expression not found from build side relation output: $output") - } - if (columnInOutput.size != 1) { - throw new IllegalStateException( - s"More than one key $expression found from build side relation output: $output") - } - val boundReference = columnInOutput.head - BoundReference(boundReference._2, boundReference._1.dataType, boundReference._1.nullable) - case other => other - } - // project operatorId = context.nextOperatorId("ClickHouseBuildSideRelationProjection") val args = context.registeredFunction val columnarProjExpr = ExpressionConverter - .replaceWithExpressionTransformer(newBoundRefKey, attributeSeq = output) + .replaceWithExpressionTransformer(key, attributeSeq = output) val projExprNodeList = new java.util.ArrayList[ExpressionNode]() - projExprNodeList.add(columnarProjExpr.doTransform(args)) + columnarProjExpr.foreach(e => projExprNodeList.add(e.doTransform(args))) PlanBuilder.makePlan( context, Lists.newArrayList( RelBuilder.makeProjectRel(readRel, projExprNodeList, context, operatorId, output.size)), - Lists.newArrayList( - ConverterUtils.genColumnNameWithExprId(ConverterUtils.getAttrFromExpr(key))) + Lists.newArrayList(genColumnNameWithExprId(key, output)) ) } + + private def genColumnNameWithExprId( + key: Seq[Expression], + output: Seq[Attribute]): util.List[String] = { + key + .map { + k => + val reference = k.collectFirst { case BoundReference(ordinal, _, _) => output(ordinal) } + assert(reference.isDefined) + reference.get + } + .map(ConverterUtils.genColumnNameWithExprId) + .toList + .asJava + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala new file mode 100644 index 000000000000..35533a4b3e9c --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/vectorized/NativeExpressionEvaluator.scala @@ -0,0 +1,39 @@ +/* + * 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.gluten.vectorized + +import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.utils.ConfigUtil + +import scala.collection.JavaConverters._ + +// TODO: move CHNativeExpressionEvaluator to NativeExpressionEvaluator +/** Scala Wrapper for ExpressionEvaluatorJniWrapper */ +object NativeExpressionEvaluator { + + def updateQueryRuntimeSettings(settings: Map[String, String]): Unit = { + ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings( + ConfigUtil.serialize( + settings + .filter(t => CHConf.startWithSettingsPrefix(t._1) && t._2.nonEmpty) + .map { + case (k, v) => + (CHConf.removeSettingsPrefix(k), v) + } + .asJava)) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala index 87c6ae343d4c..706cc5f34108 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/utils/RangePartitionerBoundsGenerator.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.shuffle.utils -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.backendsapi.clickhouse.CHValidatorApi import org.apache.gluten.execution.SortExecTransformer import org.apache.gluten.expression.ExpressionConverter import org.apache.gluten.substrait.SubstraitContext @@ -261,8 +261,9 @@ object RangePartitionerBoundsGenerator { break } if ( - !ordering.child.isInstanceOf[Attribute] && !BackendsApiManager.getSettings - .supportShuffleWithProject(rangePartitioning, child) + !ordering.child.isInstanceOf[Attribute] && !CHValidatorApi.supportShuffleWithProject( + rangePartitioning, + child) ) { enableRangePartitioning = false break diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala index dd24d475340b..bea22404eafe 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/MergeTreeFileFormat.scala @@ -21,7 +21,8 @@ import org.apache.gluten.execution.datasource.GlutenFormatFactory import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.actions.Metadata import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} -import org.apache.spark.sql.execution.datasources.mergetree.DeltaMetaReader +import org.apache.spark.sql.execution.datasources.mergetree.{DeltaMetaReader, StorageMeta} +import org.apache.spark.sql.execution.datasources.v1.Write import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.types.StructType @@ -64,3 +65,28 @@ trait MergeTreeFileFormat extends FileFormat with DataSourceRegister { } } } + +object MergeTreeFileFormat { + def createWrite(outputPath: Option[String], conf: Map[String, String]): Write.MergeTreeWrite = { + Write.MergeTreeWrite + .newBuilder() + .setDatabase(conf(StorageMeta.DB)) + .setTable(conf(StorageMeta.TABLE)) + .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID)) + .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY)) + .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY)) + .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY)) + .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY)) + .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY)) + .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY)) + .setRelativePath(outputPath.map(StorageMeta.normalizeRelativePath).getOrElse("")) + .setAbsolutePath("") + .setStoragePolicy(conf(StorageMeta.POLICY)) + .build() + } + def createWrite(metadata: Metadata): Write.MergeTreeWrite = { + // we can get the output path at the driver side + val deltaMetaReader = DeltaMetaReader(metadata) + createWrite(None, deltaMetaReader.storageConf) + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala index bce43b0ef288..13a9efa35989 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/files/MergeTreeFileCommitProtocol.scala @@ -16,9 +16,9 @@ */ package org.apache.spark.sql.delta.files -import org.apache.gluten.backendsapi.clickhouse.CHConf +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings import org.apache.gluten.memory.CHThreadGroup -import org.apache.gluten.vectorized.ExpressionEvaluatorJniWrapper +import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -31,8 +31,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext import java.util.UUID -import scala.collection.JavaConverters._ - trait MergeTreeFileCommitProtocol extends FileCommitProtocol { def outputPath: String @@ -45,8 +43,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { val jobID = taskContext.getJobID.toString val taskAttemptID = taskContext.getTaskAttemptID.toString MergeTreeCommiterHelper.prepareTaskWriteInfo(jobID, taskAttemptID) - val settings = Map(CHConf.runtimeSettings("gluten.write.reserve_partition_columns") -> "true") - ExpressionEvaluatorJniWrapper.updateQueryRuntimeSettings(settings.asJava) + val settings = Map(RuntimeSettings.NATIVE_WRITE_RESERVE_PARTITION_COLUMNS.key -> "true") + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } override def newTaskTempFile( @@ -54,22 +52,22 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { dir: Option[String], ext: String): String = { - taskContext.getConfiguration.set( - "mapreduce.task.gluten.mergetree.partition", - dir.map(p => new Path(p).toUri.toString).getOrElse("")) - + val partitionStr = dir.map(p => new Path(p).toUri.toString) val bucketIdStr = ext.split("\\.").headOption.filter(_.startsWith("_")).map(_.substring(1)) - taskContext.getConfiguration.set( - "mapreduce.task.gluten.mergetree.bucketid", - bucketIdStr.getOrElse("")) + val split = taskContext.getTaskAttemptID.getTaskID.getId - val partition = dir.map(p => new Path(p).toUri.toString + "/").getOrElse("") + // The partPrefix is used to generate the part name in the MergeTree table. + // outputPath/partition-dir/bucket-id/UUID_split + val partition = partitionStr.map(_ + "/").getOrElse("") val bucket = bucketIdStr.map(_ + "/").getOrElse("") - val taskID = taskContext.getTaskAttemptID.getTaskID.getId.toString - val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$taskID" - - taskContext.getConfiguration.set("mapreduce.task.gluten.mergetree.partPrefix", partPrefix) + val partPrefix = s"$partition$bucket${UUID.randomUUID.toString}_$split" + val settings = Map( + RuntimeSettings.PART_NAME_PREFIX.key -> partPrefix, + RuntimeSettings.PARTITION_DIR.key -> partitionStr.getOrElse(""), + RuntimeSettings.BUCKET_DIR.key -> bucketIdStr.getOrElse("") + ) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) outputPath } @@ -87,4 +85,8 @@ trait MergeTreeFileCommitProtocol extends FileCommitProtocol { ) new TaskCommitMessage(statuses) } + + override def abortTask(taskContext: TaskAttemptContext): Unit = { + MergeTreeCommiterHelper.resetCurrentTaskWriteInfo() + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala new file mode 100644 index 000000000000..03b45e540dc9 --- /dev/null +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/util/MergeTreePartitionUtils.scala @@ -0,0 +1,58 @@ +/* + * 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.delta.util + +import org.apache.spark.sql.catalyst.expressions.Cast +import org.apache.spark.sql.types.StringType + +import org.apache.hadoop.fs.Path + +/** + * `OptimizeTableCommandOverwrites` does not use `DelayedCommitProtocol`, so we can't use + * `DelayedCommitProtocol.parsePartitions`. This is a copied version.
TODO: Remove it. + */ +object MergeTreePartitionUtils { + + private val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" + + def parsePartitions(dir: String): Map[String, String] = { + // TODO: timezones? + // TODO: enable validatePartitionColumns? + val dateFormatter = DateFormatter() + val timestampFormatter = + TimestampFormatter(timestampPartitionPattern, java.util.TimeZone.getDefault) + val parsedPartition = + PartitionUtils + .parsePartition( + new Path(dir), + typeInference = false, + Set.empty, + Map.empty, + validatePartitionColumns = false, + java.util.TimeZone.getDefault, + dateFormatter, + timestampFormatter) + ._1 + .get + parsedPartition.columnNames + .zip( + parsedPartition.literals + .map(l => Cast(l, StringType).eval()) + .map(Option(_).map(_.toString).orNull)) + .toMap + } +} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala index 29fa0d0aba96..174493e5f4f6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarToRowExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.extension.ValidationResult import org.apache.gluten.metrics.GlutenTimeMetric -import org.apache.gluten.vectorized.CHNativeBlock import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} import org.apache.spark.broadcast.Broadcast @@ -103,10 +102,7 @@ class CHColumnarToRowRDD( logInfo(s"Skip ColumnarBatch of ${batch.numRows} rows, ${batch.numCols} cols") Iterator.empty } else { - val blockAddress = GlutenTimeMetric.millis(convertTime) { - _ => CHNativeBlock.fromColumnarBatch(batch).blockAddress() - } - CHExecUtil.getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows()) + GlutenTimeMetric.millis(convertTime)(_ => CHExecUtil.c2r(batch)) } } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala index 6a5c19a4f939..1342e250430e 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWrite.scala @@ -16,25 +16,28 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.backendsapi.clickhouse.RuntimeSettings +import org.apache.gluten.vectorized.NativeExpressionEvaluator import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec, HadoopMapReduceCommitProtocol} import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.execution.datasources.{BasicWriteTaskStats, ExecutedWriteSummary, PartitioningUtils, WriteJobDescription, WriteTaskResult} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.delta.stats.DeltaJobStatisticsTracker +import org.apache.spark.sql.execution.datasources._ import org.apache.spark.util.Utils import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobID, OutputCommitter, TaskAttemptContext, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import java.lang.reflect.Field import scala.collection.mutable +import scala.language.implicitConversions trait CHColumnarWrite[T <: FileCommitProtocol] { @@ -51,7 +54,17 @@ trait CHColumnarWrite[T <: FileCommitProtocol] { def abortTask(): Unit = { committer.abortTask(taskAttemptContext) } - def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] + def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] + + lazy val basicWriteJobStatsTracker: WriteTaskStatsTracker = description.statsTrackers + .find(_.isInstanceOf[BasicWriteJobStatsTracker]) + .map(_.newTaskInstance()) + .get + + lazy val deltaWriteJobStatsTracker: Option[DeltaJobStatisticsTracker] = + description.statsTrackers + .find(_.isInstanceOf[DeltaJobStatisticsTracker]) + .map(_.asInstanceOf[DeltaJobStatisticsTracker]) lazy val (taskAttemptContext: TaskAttemptContext, jobId: String) = { // Copied from `SparkHadoopWriterUtils.createJobID` to be compatible with multi-version @@ -84,29 +97,16 @@ trait CHColumnarWrite[T <: FileCommitProtocol] { object CreateFileNameSpec { def apply(taskContext: TaskAttemptContext, description: WriteJobDescription): FileNameSpec = { val fileCounter = 0 - val suffix = f".c$fileCounter%03d" + + val suffix = f"-c$fileCounter%03d" + description.outputWriterFactory.getFileExtension(taskContext) FileNameSpec("", suffix) } } -object CreateBasicWriteTaskStats { - def apply( - numFiles: Int, - updatedPartitions: Set[String], - numWrittenRows: Long): BasicWriteTaskStats = { - val partitionsInternalRows = updatedPartitions.map { - part => - val parts = new Array[Any](1) - parts(0) = part - new GenericInternalRow(parts) - }.toSeq - BasicWriteTaskStats( - partitions = partitionsInternalRows, - numFiles = numFiles, - numBytes = 101, - numRows = numWrittenRows) - } +// More details in local_engine::FileNameGenerator in NormalFileWriter.cpp +object FileNamePlaceHolder { + val ID = "{id}" + val BUCKET = "{bucket}" } /** [[HadoopMapReduceAdapter]] for [[HadoopMapReduceCommitProtocol]]. */ @@ -139,12 +139,105 @@ case class HadoopMapReduceAdapter(sparkCommitter: HadoopMapReduceCommitProtocol) GetFilename.invoke(sparkCommitter, taskContext, spec).asInstanceOf[String] } - def getTaskAttemptTempPathAndFilename( + def getTaskAttemptTempPathAndFilePattern( taskContext: TaskAttemptContext, description: WriteJobDescription): (String, String) = { val stageDir = newTaskAttemptTempPath(description.path) - val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description)) - (stageDir, filename) + + if (isBucketWrite(description)) { + val filePart = getFilename(taskContext, FileNameSpec("", "")) + val fileSuffix = CreateFileNameSpec(taskContext, description).suffix + (stageDir, s"${filePart}_${FileNamePlaceHolder.BUCKET}$fileSuffix") + } else { + val filename = getFilename(taskContext, CreateFileNameSpec(taskContext, description)) + (stageDir, filename) + } + } + + private def isBucketWrite(desc: WriteJobDescription): Boolean = { + // In Spark 3.2, bucketSpec is not defined, instead, it uses bucketIdExpression. + val bucketSpecField: Field = desc.getClass.getDeclaredField("bucketSpec") + bucketSpecField.setAccessible(true) + bucketSpecField.get(desc).asInstanceOf[Option[_]].isDefined + } +} + +/** + * {{{ + * val schema = + * StructType( + * StructField("filename", StringType, false) :: + * StructField("partition_id", StringType, false) :: + * StructField("record_count", LongType, false) :: Nil) + * }}} + */ +case class NativeFileWriteResult(filename: String, partition_id: String, record_count: Long) { + lazy val relativePath: String = if (partition_id == "__NO_PARTITION_ID__") { + filename + } else { + s"$partition_id/$filename" + } +} + +object NativeFileWriteResult { + implicit def apply(row: InternalRow): NativeFileWriteResult = { + NativeFileWriteResult(row.getString(0), row.getString(1), row.getLong(2)) + } +} + +case class NativeStatCompute(rows: Seq[InternalRow]) { + def apply[T](stats: Seq[T => Unit], extra: Option[InternalRow => Unit] = None)(implicit + creator: InternalRow => T): Unit = { + rows.foreach { + row => + val stat = creator(row) + stats.foreach(agg => agg(stat)) + extra.foreach(_(row)) + } + } +} + +case class NativeBasicWriteTaskStatsTracker( + description: WriteJobDescription, + basicWriteJobStatsTracker: WriteTaskStatsTracker) + extends (NativeFileWriteResult => Unit) { + private var numWrittenRows: Long = 0 + override def apply(stat: NativeFileWriteResult): Unit = { + val absolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id != "__NO_PARTITION_ID__") { + basicWriteJobStatsTracker.newPartition(new GenericInternalRow(Array[Any](stat.partition_id))) + } + basicWriteJobStatsTracker.newFile(absolutePath) + basicWriteJobStatsTracker.closeFile(absolutePath) + numWrittenRows += stat.record_count + } + private def finalStats: BasicWriteTaskStats = basicWriteJobStatsTracker + .getFinalStats(0) + .asInstanceOf[BasicWriteTaskStats] + + def result: BasicWriteTaskStats = finalStats.copy(numRows = numWrittenRows) +} + +case class FileCommitInfo(description: WriteJobDescription) + extends (NativeFileWriteResult => Unit) { + private val partitions: mutable.Set[String] = mutable.Set[String]() + private val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() + + def apply(stat: NativeFileWriteResult): Unit = { + val tmpAbsolutePath = s"${description.path}/${stat.relativePath}" + if (stat.partition_id != "__NO_PARTITION_ID__") { + partitions += stat.partition_id + val customOutputPath = + description.customPartitionLocations.get( + PartitioningUtils.parsePathFragment(stat.partition_id)) + if (customOutputPath.isDefined) { + addedAbsPathFiles(tmpAbsolutePath) = customOutputPath.get + "/" + stat.filename + } + } + } + + def result: (Set[String], Map[String, String]) = { + (partitions.toSet, addedAbsPathFiles.toMap) } } @@ -162,58 +255,40 @@ case class HadoopMapReduceCommitProtocolWrite( * initializing the native plan and collect native write files metrics for each backend. */ override def doSetupNativeTask(): Unit = { - val (writePath, writeFileName) = - adapter.getTaskAttemptTempPathAndFilename(taskAttemptContext, description) - logDebug(s"Native staging write path: $writePath and file name: $writeFileName") - BackendsApiManager.getIteratorApiInstance.injectWriteFilesTempPath(writePath, writeFileName) + val (writePath, writeFilePattern) = + adapter.getTaskAttemptTempPathAndFilePattern(taskAttemptContext, description) + logDebug(s"Native staging write path: $writePath and file pattern: $writeFilePattern") + + val settings = + Map( + RuntimeSettings.TASK_WRITE_TMP_DIR.key -> writePath, + RuntimeSettings.TASK_WRITE_FILENAME_PATTERN.key -> writeFilePattern) + NativeExpressionEvaluator.updateQueryRuntimeSettings(settings) } - def doCollectNativeResult(cb: ColumnarBatch): Option[WriteTaskResult] = { - val numFiles = cb.numRows() + def doCollectNativeResult(stats: Seq[InternalRow]): Option[WriteTaskResult] = { // Write an empty iterator - if (numFiles == 0) { + if (stats.isEmpty) { None } else { - val file_col = cb.column(0) - val partition_col = cb.column(1) - val count_col = cb.column(2) - - val outputPath = description.path - val partitions: mutable.Set[String] = mutable.Set[String]() - val addedAbsPathFiles: mutable.Map[String, String] = mutable.Map[String, String]() - - var numWrittenRows: Long = 0 - Range(0, cb.numRows()).foreach { - i => - val targetFileName = file_col.getUTF8String(i).toString - val partition = partition_col.getUTF8String(i).toString - if (partition != "__NO_PARTITION_ID__") { - partitions += partition - val tmpOutputPath = outputPath + "/" + partition + "/" + targetFileName - val customOutputPath = - description.customPartitionLocations.get( - PartitioningUtils.parsePathFragment(partition)) - if (customOutputPath.isDefined) { - addedAbsPathFiles(tmpOutputPath) = customOutputPath.get + "/" + targetFileName - } - } - numWrittenRows += count_col.getLong(i) - } - - val updatedPartitions = partitions.toSet - val summary = - ExecutedWriteSummary( - updatedPartitions = updatedPartitions, - stats = Seq(CreateBasicWriteTaskStats(numFiles, updatedPartitions, numWrittenRows))) + val commitInfo = FileCommitInfo(description) + val basicNativeStat = NativeBasicWriteTaskStatsTracker(description, basicWriteJobStatsTracker) + val basicNativeStats = Seq(commitInfo, basicNativeStat) + NativeStatCompute(stats)(basicNativeStats) + val (partitions, addedAbsPathFiles) = commitInfo.result + val updatedPartitions = partitions Some( WriteTaskResult( - new TaskCommitMessage(addedAbsPathFiles.toMap -> updatedPartitions), - summary)) + new TaskCommitMessage(addedAbsPathFiles -> updatedPartitions), + ExecutedWriteSummary( + updatedPartitions = updatedPartitions, + stats = Seq(basicNativeStat.result)) + )) } } - override def commitTask(batch: ColumnarBatch): Option[WriteTaskResult] = { - doCollectNativeResult(batch).map( + override def commitTask(writeResults: Seq[InternalRow]): Option[WriteTaskResult] = { + doCollectNativeResult(writeResults).map( nativeWriteTaskResult => { val (_, taskCommitTime) = Utils.timeTakenMs { committer.commitTask(taskAttemptContext) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala index bf051671fbba..503fd1a90caa 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/CHColumnarWriteFilesExec.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.memory.CHThreadGroup + import org.apache.spark.{Partition, SparkException, TaskContext, TaskOutputFileAlreadyExistException} import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} import org.apache.spark.rdd.RDD @@ -25,6 +27,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.write.WriterCommitMessage import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils @@ -46,14 +49,16 @@ class CHColumnarWriteFilesRDD( extends RDD[WriterCommitMessage](prev) { private def reportTaskMetrics(writeTaskResult: WriteTaskResult): Unit = { - val stats = writeTaskResult.summary.stats.head.asInstanceOf[BasicWriteTaskStats] - val (numBytes, numWrittenRows) = (stats.numBytes, stats.numRows) - // Reports bytesWritten and recordsWritten to the Spark output metrics. - // We should update it after calling `commitTask` to overwrite the metrics. - Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { - outputMetrics => - outputMetrics.setBytesWritten(numBytes) - outputMetrics.setRecordsWritten(numWrittenRows) + writeTaskResult.summary.stats.find(_.isInstanceOf[BasicWriteTaskStats]).foreach { + s => + val stats = s.asInstanceOf[BasicWriteTaskStats] + // Reports bytesWritten and recordsWritten to the Spark output metrics. + // We should update it after calling `commitTask` to overwrite the metrics. + Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { + outputMetrics => + outputMetrics.setBytesWritten(stats.numBytes) + outputMetrics.setRecordsWritten(stats.numRows) + } } } @@ -78,6 +83,7 @@ class CHColumnarWriteFilesRDD( } override def compute(split: Partition, context: TaskContext): Iterator[WriterCommitMessage] = { + CHThreadGroup.registerNewThreadGroup() val commitProtocol = CHColumnarWrite(jobTrackerID, description, committer) commitProtocol.setupTask() @@ -90,8 +96,19 @@ class CHColumnarWriteFilesRDD( assert(iter.hasNext) val resultColumnarBatch = iter.next() assert(resultColumnarBatch != null) + + /** + * we assume the number of records is less than 10,000.So the memory overhead is acceptable. + * otherwise, we need to access ColumnarBatch row by row, which is not efficient. + */ + val writeResults = CHExecUtil.c2r(resultColumnarBatch).map(_.copy()).toSeq + // TODO: we need close iterator here before processing the result. + // TODO: task commit time + // TODO: get the schema from result ColumnarBatch and verify it. + assert(!iter.hasNext) + val writeTaskResult = commitProtocol - .commitTask(resultColumnarBatch) + .commitTask(writeResults) .orElse({ // If we are writing an empty iterator, then gluten backend would do nothing. // Here we fallback to use vanilla Spark write files to generate an empty file for diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala index b360e286e504..7f80c134e5d6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHFormatWriterInjects.scala @@ -78,7 +78,7 @@ trait CHFormatWriterInjects extends GlutenFormatWriterInjectsBase { val datasourceJniWrapper = new CHDatasourceJniWrapper(outputPath, createWriteRel(outputPath, dataSchema, context)) - new FakeRowOutputWriter(datasourceJniWrapper, outputPath) + new FakeRowOutputWriter(Some(datasourceJniWrapper), outputPath) } override def inferSchema( diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala index 76f4b467ded2..3e0e393459a6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/CHMergeTreeWriterInjects.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.v1 +import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution.ColumnarToRowExecBase import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.substrait.`type`.ColumnTypeNode @@ -26,6 +27,7 @@ import org.apache.gluten.substrait.rel.RelBuilder import org.apache.gluten.utils.ConfigUtil import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.delta.MergeTreeFileFormat import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeRowAdaptor, OutputWriter} @@ -75,22 +77,7 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects { .setFormat(formatName) .setJobTaskAttemptId(s"$jobID/$taskAttemptID") .build()) - .setMergetree( - Write.MergeTreeWrite - .newBuilder() - .setDatabase(conf(StorageMeta.DB)) - .setTable(conf(StorageMeta.TABLE)) - .setSnapshotId(conf(StorageMeta.SNAPSHOT_ID)) - .setOrderByKey(conf(StorageMeta.ORDER_BY_KEY)) - .setLowCardKey(conf(StorageMeta.LOW_CARD_KEY)) - .setMinmaxIndexKey(conf(StorageMeta.MINMAX_INDEX_KEY)) - .setBfIndexKey(conf(StorageMeta.BF_INDEX_KEY)) - .setSetIndexKey(conf(StorageMeta.SET_INDEX_KEY)) - .setPrimaryKey(conf(StorageMeta.PRIMARY_KEY)) - .setRelativePath(StorageMeta.normalizeRelativePath(outputPath)) - .setAbsolutePath("") - .setStoragePolicy(conf(StorageMeta.POLICY)) - .build()) + .setMergetree(MergeTreeFileFormat.createWrite(Some(outputPath), conf)) .build() } @@ -100,14 +87,24 @@ class CHMergeTreeWriterInjects extends CHFormatWriterInjects { context: TaskAttemptContext, nativeConf: ju.Map[String, String]): OutputWriter = { - val datasourceJniWrapper = new CHDatasourceJniWrapper( - context.getConfiguration.get("mapreduce.task.gluten.mergetree.partPrefix"), - context.getConfiguration.get("mapreduce.task.gluten.mergetree.partition"), - context.getConfiguration.get("mapreduce.task.gluten.mergetree.bucketid"), - createWriteRel(outputPath, dataSchema, context), - ConfigUtil.serialize(nativeConf) - ) - new FakeRowOutputWriter(datasourceJniWrapper, outputPath) + val wrapper = if (CHConf.get.enableOnePipelineMergeTreeWrite) { + + /** + * In pipeline mode, CHColumnarWriteFilesRDD.writeFilesForEmptyIterator will create a JNI + * wrapper which is not needed in this case. + * + * TODO: We should refactor the code to avoid creating the JNI wrapper in this case. + */ + None + } else { + val datasourceJniWrapper = new CHDatasourceJniWrapper( + createWriteRel(outputPath, dataSchema, context), + ConfigUtil.serialize(nativeConf) + ) + Some(datasourceJniWrapper) + } + + new FakeRowOutputWriter(wrapper, outputPath) } override val formatName: String = "mergetree" diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala index 9ebce9d3b455..574c9784257d 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/FakeRowOutputWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.execution.datasources.{CHDatasourceJniWrapper, FakeR import scala.collection.mutable.ArrayBuffer -class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPath: String) +class FakeRowOutputWriter(datasourceJniWrapper: Option[CHDatasourceJniWrapper], outputPath: String) extends OutputWriter { protected var addFiles: ArrayBuffer[AddFile] = new ArrayBuffer[AddFile]() @@ -35,12 +35,12 @@ class FakeRowOutputWriter(datasourceJniWrapper: CHDatasourceJniWrapper, outputPa if (nextBatch.numRows > 0) { val col = nextBatch.column(0).asInstanceOf[CHColumnVector] - datasourceJniWrapper.write(col.getBlockAddress) + datasourceJniWrapper.foreach(_.write(col.getBlockAddress)) } // else ignore this empty block } override def close(): Unit = { - datasourceJniWrapper.close() + datasourceJniWrapper.foreach(_.close()) } // Do NOT add override keyword for compatibility on spark 3.1. diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala index 1d6a488afebf..436c6fa66a29 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/clickhouse/MergeTreeCommiterHelper.scala @@ -48,4 +48,8 @@ object MergeTreeCommiterHelper { require(currentTaskWriteInfo.get() == null, "currentTaskWriteInfo is not null") currentTaskWriteInfo.set(TaskWriteInfo(s"$jobID/$taskAttemptID")) } + + def resetCurrentTaskWriteInfo(): Unit = { + currentTaskWriteInfo.remove() + } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala index 8c610929a3d8..c4c971633a11 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/metadata/AddFileTags.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.util.MergeTreePartitionUtils import org.apache.spark.sql.execution.datasources.clickhouse.WriteReturnedMetric import com.fasterxml.jackson.core.`type`.TypeReference @@ -26,7 +27,6 @@ import org.apache.hadoop.fs.Path import java.util.{List => JList} import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer @SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) class AddMergeTreeParts( @@ -192,46 +192,47 @@ object AddFileTags { tableName: String, originPathStr: String, returnedMetrics: String, - hostName: Seq[String]): ArrayBuffer[AddFile] = { + hostName: Seq[String]): Seq[AddFile] = { + val mapper: ObjectMapper = new ObjectMapper() - try { - val values: JList[WriteReturnedMetric] = - mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {}) - var addFiles = new ArrayBuffer[AddFile]() - val path = new Path(originPathStr) - val modificationTime = System.currentTimeMillis() - addFiles.appendAll(values.asScala.map { - value => - AddFileTags.partsInfoToAddFile( - database, - tableName, - "MergeTree", - path.toUri.getPath, - hostName.map(_.trim).mkString(","), - value.getPartName, - "", - value.getRowCount, - value.getDiskSize, - -1L, - -1L, - modificationTime, - "", - -1L, - -1L, - -1, - -1L, - value.getBucketId, - path.toString, - dataChange = true, - "", - partitionValues = value.getPartitionValues.asScala.toMap, - marks = value.getMarkCount - ) - }) - addFiles - } catch { - case e: Exception => - ArrayBuffer.empty[AddFile] - } + val values: JList[WriteReturnedMetric] = + mapper.readValue(returnedMetrics, new TypeReference[JList[WriteReturnedMetric]]() {}) + val path = new Path(originPathStr) + val modificationTime = System.currentTimeMillis() + + values.asScala.map { + value => + val partitionValues = if (value.getPartitionValues.isEmpty) { + Map.empty[String, String] + } else { + MergeTreePartitionUtils.parsePartitions(value.getPartitionValues) + } + + AddFileTags.partsInfoToAddFile( + database, + tableName, + "MergeTree", + path.toUri.getPath, + hostName.map(_.trim).mkString(","), + value.getPartName, + "", + value.getRowCount, + value.getDiskSize, + -1L, + -1L, + modificationTime, + "", + -1L, + -1L, + -1, + -1L, + value.getBucketId, + path.toString, + dataChange = true, + "", + partitionValues = partitionValues, + marks = value.getMarkCount + ) + }.toSeq } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala index 92887f16d70a..668525ba0a40 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/joins/ClickHouseBuildSideRelation.scala @@ -22,8 +22,8 @@ import org.apache.gluten.vectorized._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection} +import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, IdentityBroadcastMode} import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.CHShuffleReadStreamFactory @@ -72,7 +72,7 @@ case class ClickHouseBuildSideRelation( } /** - * Transform columnar broadcast value to Array[InternalRow] by key and distinct. + * Transform columnar broadcast value to Array[InternalRow] by key. * * @return */ @@ -80,10 +80,18 @@ case class ClickHouseBuildSideRelation( // native block reader val blockReader = new CHStreamReader(CHShuffleReadStreamFactory.create(batches, true)) val broadCastIter: Iterator[ColumnarBatch] = IteratorUtil.createBatchIterator(blockReader) + + val transformProjections = mode match { + case HashedRelationBroadcastMode(k, _) => k + case IdentityBroadcastMode => output + } + // Expression compute, return block iterator val expressionEval = new SimpleExpressionEval( new ColumnarNativeIterator(broadCastIter.asJava), - PlanNodesUtil.genProjectionsPlanNode(key, output)) + PlanNodesUtil.genProjectionsPlanNode(transformProjections, output)) + + val proj = UnsafeProjection.create(Seq(key)) try { // convert columnar to row @@ -95,6 +103,7 @@ case class ClickHouseBuildSideRelation( } else { CHExecUtil .getRowIterFromSparkRowInfo(block, batch.numColumns(), batch.numRows()) + .map(proj) .map(row => row.copy()) } }.toArray diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala index 6d91108d22da..cc172ac4b543 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/utils/CHExecUtil.scala @@ -113,12 +113,8 @@ object CHExecUtil extends Logging { iter => iter.flatMap( batch => { - val blockAddress = CHNativeBlock.fromColumnarBatch(batch).blockAddress() - // generate rows from a columnar batch - val rowItr: Iterator[InternalRow] = - getRowIterFromSparkRowInfo(blockAddress, batch.numCols(), batch.numRows()) - + val rowItr: Iterator[InternalRow] = c2r(batch) val projection = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) val mutablePair = new MutablePair[InternalRow, Null]() @@ -164,6 +160,14 @@ object CHExecUtil extends Logging { val rowInfo = CHBlockConverterJniWrapper.convertColumnarToRow(blockAddress, null) getRowIterFromSparkRowInfo(rowInfo, columns, rows) } + + def c2r(batch: ColumnarBatch): Iterator[InternalRow] = { + getRowIterFromSparkRowInfo( + CHNativeBlock.fromColumnarBatch(batch).blockAddress(), + batch.numCols(), + batch.numRows()) + } + private def buildPartitionedBlockIterator( cbIter: Iterator[ColumnarBatch], options: IteratorOptions, diff --git a/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala new file mode 100644 index 000000000000..676061235ed0 --- /dev/null +++ b/backends-clickhouse/src/test-iceberg/scala/org/apache/gluten/execution/iceberg/ClickHouseIcebergSuite.scala @@ -0,0 +1,640 @@ +/* + * 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.gluten.execution.iceberg + +import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, IcebergScanTransformer} +import org.apache.gluten.GlutenConfig +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row + +class ClickHouseIcebergSuite extends GlutenClickHouseWholeStageTransformerSuite { + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.sql.shuffle.partitions", "2") + .set("spark.memory.offHeap.size", "2g") + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set("spark.sql.autoBroadcastJoinThreshold", "-1") + .set( + "spark.sql.extensions", + "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions") + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkCatalog") + .set("spark.sql.catalog.spark_catalog.type", "hadoop") + .set("spark.sql.catalog.spark_catalog.warehouse", s"file://$basePath/tpch-data-iceberg") + } + + test("iceberg transformer exists") { + withTable("iceberg_tb") { + spark.sql( + """ + |create table iceberg_tb using iceberg as + |(select 1 as col1, 2 as col2, 3 as col3) + |""".stripMargin) + + runQueryAndCompare( + """ + |select * from iceberg_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p string) + |using iceberg + |partitioned by (bucket(4, id)); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 'p4'), + |(1, 'a1', 'p1'), + |(2, 'a3', 'p2'), + |(1, 'a2', 'p1'), + |(3, 'a4', 'p3'); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id)); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 23), + |(1, 'b2', 21), + |(4, 'b5', 24), + |(2, 'b3', 22), + |(1, 'b1', 21); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) + case _ => // do nothing + } + checkLengthAndPlan(df, 7) + } + } + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join with partition", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 2), + |(1, 'a1', 1), + |(2, 'a3', 1), + |(1, 'a2', 1), + |(3, 'a4', 2); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 2), + |(1, 'b2', 1), + |(4, 'b5', 2), + |(2, 'b3', 1), + |(1, 'b1', 1); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id and s.p = i.p; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) + case _ => // do nothing + } + checkLengthAndPlan(df, 7) + } + } + } + } + } + + testWithSpecifiedSparkVersion( + "iceberg bucketed join with partition filter", Array("3.3", "3.5")) { + val leftTable = "p_str_tb" + val rightTable = "p_int_tb" + withTable(leftTable, rightTable) { + // Partition key of string type. + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $leftTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $leftTable values + |(4, 'a5', 2), + |(1, 'a1', 1), + |(2, 'a3', 1), + |(1, 'a2', 1), + |(3, 'a4', 2); + |""".stripMargin + ) + } + + // Partition key of integer type. + withSQLConf( + GlutenConfig.GLUTEN_ENABLED_KEY -> "false" + ) { + // Gluten does not support write iceberg table. + spark.sql( + s""" + |create table $rightTable(id int, name string, p int) + |using iceberg + |partitioned by (bucket(4, id), p); + |""".stripMargin) + spark.sql( + s""" + |insert into table $rightTable values + |(3, 'b4', 2), + |(1, 'b2', 1), + |(4, 'b5', 2), + |(2, 'b3', 1), + |(1, 'b1', 1); + |""".stripMargin + ) + } + + withSQLConf( + "spark.sql.sources.v2.bucketing.enabled" -> "true", + "spark.sql.requireAllClusterKeysForCoPartition" -> "false", + "spark.sql.adaptive.enabled" -> "false", + "spark.sql.iceberg.planning.preserve-data-grouping" -> "true", + "spark.sql.autoBroadcastJoinThreshold" -> "-1", + "spark.sql.sources.v2.bucketing.pushPartValues.enabled" -> "true" + ) { + runQueryAndCompare( + s""" + |select s.id, s.name, i.name, i.p + | from $leftTable s inner join $rightTable i + | on s.id = i.id + | where s.p = 1 and i.p = 1; + |""".stripMargin) { + df => { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[IcebergScanTransformer] + }) == 2) + getExecutedPlan(df).map { + case plan if plan.isInstanceOf[IcebergScanTransformer] => + assert( + plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1) + case _ => // do nothing + } + checkLengthAndPlan(df, 5) + } + } + } + } + } + + testWithSpecifiedSparkVersion("iceberg: time travel") { + withTable("iceberg_tm") { + spark.sql( + s""" + |create table iceberg_tm (id int, name string) using iceberg + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_tm values (1, "v1"), (2, "v2") + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_tm values (3, "v3"), (4, "v4") + |""".stripMargin) + + val df = + spark.sql("select snapshot_id from default.iceberg_tm.snapshots where parent_id is null") + val value = df.collectAsList().get(0).getAs[Long](0); + spark.sql(s"call system.set_current_snapshot('default.iceberg_tm',$value)"); + val data = runQueryAndCompare("select * from iceberg_tm") { _ => } + checkLengthAndPlan(data, 2) + checkAnswer(data, Row(1, "v1") :: Row(2, "v2") :: Nil) + } + } + + test("iceberg: partition filters") { + withTable("iceberg_pf") { + spark.sql( + s""" + |create table iceberg_pf (id int, name string) + | using iceberg partitioned by (name) + |""".stripMargin) + spark.sql( + s""" + |insert into iceberg_pf values (1, "v1"), (2, "v2"), (3, "v1"), (4, "v2") + |""".stripMargin) + val df1 = runQueryAndCompare("select * from iceberg_pf where name = 'v1'") { _ => } + checkLengthAndPlan(df1, 2) + checkAnswer(df1, Row(1, "v1") :: Row(3, "v1") :: Nil) + } + } + + test("iceberg read mor table - delete and update") { + withTable("iceberg_mor_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'), + | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + // Delete row again. + spark.sql( + """ + |delete from iceberg_mor_tb where id = 6; + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // TODO: support merge-on-read mode + ignore("iceberg read mor table - delete and update with merge-on-read mode") { + withTable("iceberg_mor_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2', + | 'write.delete.mode' = 'merge-on-read', + | 'write.update.mode' = 'merge-on-read', + | 'write.merge.mode' = 'merge-on-read' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'), + | (4, 'a4', 'p1'), (5, 'a5', 'p2'), (6, 'a6', 'p1'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + // Delete row again. + spark.sql( + """ + |delete from iceberg_mor_tb where id = 6; + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + test("iceberg read mor table - merge into") { + withTable("iceberg_mor_tb", "merge_into_source_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2' + |) + |partitioned by (p); + |""".stripMargin) + spark.sql( + """ + |create table merge_into_source_tb ( + | id int, + | name string, + | p string + |) using iceberg; + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'); + |""".stripMargin) + spark.sql( + """ + |insert into table merge_into_source_tb + |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'), + | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + + // Merge into. + spark.sql( + """ + |merge into iceberg_mor_tb t + |using (select * from merge_into_source_tb) s + |on t.id = s.id + |when matched then + | update set t.name = s.name, t.p = s.p + |when not matched then + | insert (id, name, p) values (s.id, s.name, s.p); + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // TODO: support merge-on-read mode + ignore("iceberg read mor table - merge into with merge-on-read mode") { + withTable("iceberg_mor_tb", "merge_into_source_tb") { + withSQLConf(GlutenConfig.GLUTEN_ENABLED_KEY -> "false") { + spark.sql( + """ + |create table iceberg_mor_tb ( + | id int, + | name string, + | p string + |) using iceberg + |tblproperties ( + | 'format-version' = '2', + | 'write.delete.mode' = 'merge-on-read', + | 'write.update.mode' = 'merge-on-read', + | 'write.merge.mode' = 'merge-on-read' + |) + |partitioned by (p); + |""".stripMargin) + spark.sql( + """ + |create table merge_into_source_tb ( + | id int, + | name string, + | p string + |) using iceberg; + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table iceberg_mor_tb + |values (1, 'a1', 'p1'), (2, 'a2', 'p1'), (3, 'a3', 'p2'); + |""".stripMargin) + spark.sql( + """ + |insert into table merge_into_source_tb + |values (1, 'a1_1', 'p2'), (2, 'a2_1', 'p2'), (3, 'a3_1', 'p1'), + | (4, 'a4', 'p2'), (5, 'a5', 'p1'), (6, 'a6', 'p2'); + |""".stripMargin) + + // Delete row. + spark.sql( + """ + |delete from iceberg_mor_tb where name = 'a1'; + |""".stripMargin + ) + // Update row. + spark.sql( + """ + |update iceberg_mor_tb set name = 'new_a2' where id = 'a2'; + |""".stripMargin + ) + + // Merge into. + spark.sql( + """ + |merge into iceberg_mor_tb t + |using (select * from merge_into_source_tb) s + |on t.id = s.id + |when matched then + | update set t.name = s.name, t.p = s.p + |when not matched then + | insert (id, name, p) values (s.id, s.name, s.p); + |""".stripMargin + ) + } + runQueryAndCompare( + """ + |select * from iceberg_mor_tb; + |""".stripMargin) { + checkGlutenOperatorMatch[IcebergScanTransformer] + } + } + } + + // Spark configuration spark.sql.iceberg.handle-timestamp-without-timezone is not supported + // in Spark 3.4 + // TODO: there is a bug when using timestamp type as the partition column + testWithSpecifiedSparkVersion("iceberg partition type - timestamp", Array("")) { + Seq("true", "false").foreach { + flag => + withSQLConf( + "spark.sql.iceberg.handle-timestamp-without-timezone" -> flag, + "spark.sql.iceberg.use-timestamp-without-timezone-in-new-tables" -> flag) { + withTable("part_by_timestamp") { + spark.sql( + """ + |create table part_by_timestamp ( + | p timestamp + |) using iceberg + |tblproperties ( + | 'format-version' = '1' + |) + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql( + """ + |insert into table part_by_timestamp + |values (TIMESTAMP '2022-01-01 00:01:20'); + |""".stripMargin) + val df = spark.sql("select * from part_by_timestamp") + checkAnswer(df, Row(java.sql.Timestamp.valueOf("2022-01-01 00:01:20")) :: Nil) + } + } + } + } +} diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala similarity index 62% rename from backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala rename to backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala index 0999121dd8bd..5d9f761e8a74 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxOrcWriterInjects.scala +++ b/backends-clickhouse/src/test/delta-20/org/apache/spark/gluten/delta/DeltaStatsUtils.scala @@ -14,19 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.velox +package org.apache.spark.gluten.delta -import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.mutable +import org.apache.spark.sql.{DataFrame, SparkSession} -class VeloxOrcWriterInjects extends VeloxFormatWriterInjects { - override def nativeConf( - options: Map[String, String], - compressionCodec: String): java.util.Map[String, String] = { - val sparkOptions = new mutable.HashMap[String, String]() - // TODO: implement it - sparkOptions.asJava - } +object DeltaStatsUtils { - override val formatName: String = "orc" + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + throw new IllegalAccessException("Method not used below spark 3.5") + } } diff --git a/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.scala new file mode 100644 index 000000000000..5d9f761e8a74 --- /dev/null +++ b/backends-clickhouse/src/test/delta-23/org/apache/spark/gluten/delta/DeltaStatsUtils.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.gluten.delta + +import org.apache.spark.sql.{DataFrame, SparkSession} + +object DeltaStatsUtils { + + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + throw new IllegalAccessException("Method not used below spark 3.5") + } +} diff --git a/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala new file mode 100644 index 000000000000..08681892dc1d --- /dev/null +++ b/backends-clickhouse/src/test/delta-32/org/apache/spark/gluten/delta/DeltaStatsUtils.scala @@ -0,0 +1,118 @@ +/* + * 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.gluten.delta + +import org.apache.spark.sql.{wrapper, DataFrame, SparkSession} +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, DeltaLog, DeltaLogFileIndex, NoMapping} +import org.apache.spark.sql.delta.actions.Protocol +import org.apache.spark.sql.delta.actions.TableFeatureProtocolUtils.TABLE_FEATURES_MIN_WRITER_VERSION +import org.apache.spark.sql.delta.stats.{DeltaStatsColumnSpec, StatisticsCollection} +import org.apache.spark.sql.functions.{col, from_json, regexp_replace} +import org.apache.spark.sql.types._ + +import org.apache.hadoop.fs.{FileSystem, Path} + +import java.util.Locale + +case class Statistics(override val tableSchema: StructType) extends StatisticsCollection { + override val outputAttributeSchema: StructType = tableSchema + // [[outputTableStatsSchema]] is the candidate schema to find statistics columns. + override val outputTableStatsSchema: StructType = tableSchema + override val statsColumnSpec = DeltaStatsColumnSpec(None, Some(32)) + override val columnMappingMode: DeltaColumnMappingMode = NoMapping + override val protocol: Protocol = Protocol( + minReaderVersion = 1, + minWriterVersion = TABLE_FEATURES_MIN_WRITER_VERSION, + readerFeatures = None, + writerFeatures = Some(Set())) + + override def spark: SparkSession = { + throw new Exception("Method not used in statisticsCollectionFromMetadata") + } +} + +object DeltaStatsUtils { + + private def stringToDataType(dataType: String): DataType = + dataType.toLowerCase(Locale.ROOT) match { + case "bigint" => LongType + case "double" => DoubleType + case "string" => StringType + case "date" => DateType + case _ => throw new IllegalArgumentException(s"Unsupported data type: $dataType") + } + + /** + * Parse a schema string as follows into a [[StructType]]. + * {{{ + * l_orderkey bigint, + * l_partkey bigint, + * l_suppkey bigint, + * l_linenumber bigint + * }}} + */ + private def stringToSchema(schemaString: String): StructType = { + val fields = schemaString.trim.split(",\\s*").map { + fieldString => + val parts = fieldString.trim.split("\\s+") + require(parts.length == 2, s"Invalid field definition: $fieldString") + val fieldName = parts(0).trim + val fieldType = stringToDataType(parts(1).trim) + StructField(fieldName, fieldType, nullable = true) + } + StructType(fields) + } + + def statsDF( + sparkSession: SparkSession, + deltaJson: String, + schema: String + ): DataFrame = { + + val statistics = Statistics(stringToSchema(schema)) + + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(Map.empty) + wrapper + .ofRows( + sparkSession, + DeltaLog.indexToRelation( + sparkSession, + DeltaLogFileIndex( + DeltaLogFileIndex.COMMIT_FILE_FORMAT, + FileSystem.get(hadoopConf), + Seq(new Path(deltaJson))), + Map.empty) + ) + .select("add") + .filter("add is not null") + .withColumns(Map( + "path" -> regexp_replace( + col("add.path"), + "-[0-9A-Fa-f]{8}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{4}-[0-9A-Fa-f]{12}", + "" + ), // normalize file name + "stats" -> from_json(col("add.stats"), statistics.statsSchema) + )) + .select( + "path", + "stats.numRecords", + "stats.minValues.*", + "stats.maxValues.*", + "stats.nullCount.*") + } + +} diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala index 2663a8fb81fb..9988d5372943 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDecimalSuite.scala @@ -404,6 +404,18 @@ class GlutenClickHouseDecimalSuite ) } + test("GLUTEN-8074 Fix adjust output constant column") { + val sql = + s""" + |select bround(1002.5786, 3), + | bround(-10.8, 0), + | bround(13.888888888889, 5) + |from $decimalTable + |WHERE bround(cast(decimal_field as decimal(30, 2)), 1) > 0 LIMIT 2; + |""".stripMargin + runQueryAndCompare(sql)(_ => {}) + } + test("fix decimal32 with negative value") { val schema = StructType.apply( Seq( diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala index 8871aaec18f5..2f55510a7b1f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala @@ -16,7 +16,10 @@ */ package org.apache.gluten.execution +import org.apache.gluten.GlutenConfig + import org.apache.spark.SparkConf +import org.apache.spark.gluten.delta.DeltaStatsUtils import org.apache.spark.sql.SaveMode import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper @@ -49,7 +52,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") - .set("spark.gluten.sql.native.writer.enabled", "true") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, spark35.toString) .set("spark.sql.storeAssignmentPolicy", "legacy") .setCHSettings("mergetree.merge_after_insert", false) .set("spark.databricks.delta.retentionDurationCheck.enabled", "false") @@ -59,67 +62,45 @@ class GlutenClickHouseDeltaParquetWriteSuite createNotNullTPCHTablesInParquet(tablesPath) } + private val q1SchemaString: String = + s""" l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string""".stripMargin + + private def doInsert(drop: String, create: String, insert: String): Unit = { + spark.sql(drop) + spark.sql(create) + spark.sql(insert) + } + private def drop(table: String): String = s"DROP TABLE IF EXISTS $table" + private def createLineitem(table: String): String = + s"""CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta + |TBLPROPERTIES (write.format.default = 'parquet') + |LOCATION '$basePath/$table' + |""".stripMargin + test("test parquet table write with the delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet; - |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |TBLPROPERTIES (write.format.default = 'parquet') - |LOCATION '$basePath/lineitem_delta_parquet' - |""".stripMargin) + def insert(table: String): String = + s"insert into table $table select /*+ REPARTITION(5) */ * from lineitem" - spark.sql(s""" - | insert into table lineitem_delta_parquet - | select /*+ REPARTITION(5) */ * from lineitem - |""".stripMargin) + val table = "lineitem_delta_parquet" + doInsert(drop(table), createLineitem(table), insert(table)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(table)) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -128,112 +109,70 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(parquetScan.nodeName.startsWith("Scan parquet ")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet ")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) assert(addFiles.size === 5) } + + if (spark35) { + val vanillaTable = "lineitem_delta_parquet_vanilla" + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { + doInsert(drop(vanillaTable), createLineitem(vanillaTable), insert(vanillaTable)) + } + val expected = DeltaStatsUtils + .statsDF( + spark, + s"$basePath/$vanillaTable/_delta_log/00000000000000000001.json", + q1SchemaString) + .collect() + + checkAnswer( + DeltaStatsUtils.statsDF( + spark, + s"$basePath/$table/_delta_log/00000000000000000001.json", + q1SchemaString), + expected + ) + } } test("test parquet insert overwrite with the delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite; - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite' - |""".stripMargin) - - spark.sql(s""" - | insert into table lineitem_delta_parquet_insertoverwrite - | select * from lineitem - |""".stripMargin) + def insert(table: String): String = + s"insert into table $table select * from lineitem" + val table = "lineitem_delta_parquet_insertoverwrite" + doInsert(drop(table), createLineitem(table), insert(table)) spark.sql(s""" - | insert overwrite table lineitem_delta_parquet_insertoverwrite + | insert overwrite table $table | select * from lineitem where mod(l_orderkey,2) = 1 |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_delta_parquet_insertoverwrite - |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 300001 - ) + + // total rows should remain unchanged + assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 300001) } test("test parquet insert overwrite partitioned table with small table, static with delta") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite2; - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite2 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |PARTITIONED BY (l_shipdate) - |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite2' - |""".stripMargin) - - spark.sql(s""" - | insert into table lineitem_delta_parquet_insertoverwrite2 - | select * from lineitem - | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31' - |""".stripMargin) - - spark.sql( + val table = "lineitem_delta_parquet_insertoverwrite2" + doInsert( + drop(table), s""" - | insert overwrite table lineitem_delta_parquet_insertoverwrite2 - | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' - |""".stripMargin) - val sql2 = + |CREATE TABLE IF NOT EXISTS $table ($q1SchemaString) USING delta + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/$table' + |""".stripMargin, s""" - | select count(*) from lineitem_delta_parquet_insertoverwrite2 - | + | insert into table $table select * from lineitem + | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-03-31' |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 2418 ) + spark.sql(s""" + | insert overwrite table $table select * from lineitem + | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + // total rows should remain unchanged + assert(spark.sql(s"select count(*) from $table").collect().apply(0).get(0) === 2418) } test("test parquet insert overwrite partitioned table with small table, dynamic with delta") { @@ -244,24 +183,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite3 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |PARTITIONED BY (l_shipdate) |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite3' @@ -297,24 +219,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_update - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_update' |""".stripMargin) @@ -348,7 +253,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -360,10 +265,9 @@ class GlutenClickHouseDeltaParquetWriteSuite | select count(*) from lineitem_delta_parquet_update | |""".stripMargin - assert( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) === 600572 - ) + + // total rows should remain unchanged + assert(spark.sql(sql2).collect().apply(0).get(0) === 600572) } test("test parquet table delete with the delta") { @@ -373,24 +277,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_delete' |""".stripMargin) @@ -440,24 +327,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_upsert - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |LOCATION '$basePath/lineitem_delta_parquet_upsert' |""".stripMargin) @@ -542,24 +412,7 @@ class GlutenClickHouseDeltaParquetWriteSuite spark.sql(s""" |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_partition - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) + |($q1SchemaString) |USING delta |PARTITIONED BY (l_shipdate, l_returnflag) |LOCATION '$basePath/lineitem_delta_parquet_partition' @@ -635,32 +488,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | l_comment from lineitem | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_partition - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_partition"), compareResult = false) { df => val result = df.collect() assert(result.length === 2) @@ -678,7 +506,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) assert(parquetScan.metrics("numFiles").value === 201) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -703,32 +531,7 @@ class GlutenClickHouseDeltaParquetWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_ctas1 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas1")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -736,7 +539,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -756,65 +559,14 @@ class GlutenClickHouseDeltaParquetWriteSuite |LOCATION '$basePath/lineitem_mergetree_ctas2' | as select * from lineitem |""".stripMargin) - - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_delta_parquet_ctas2 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } - + runTPCHQueryBySQL(1, q1("lineitem_delta_parquet_ctas2")) { _ => {} } } test("test path based parquet write with the delta") { val dataPath = s"$basePath/lineitem_delta_parquet_filebased" clearDataPath(dataPath) - - val sourceDF = spark.sql(s""" - |select * from lineitem - |""".stripMargin) - - spark.sql(s""" - |CREATE TABLE delta.`$dataPath` ( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING delta - |""".stripMargin) + val sourceDF = spark.sql("select * from lineitem") + spark.sql(s"CREATE TABLE delta.`$dataPath` ($q1SchemaString) USING delta") sourceDF.write .format("delta") @@ -829,32 +581,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .mode(SaveMode.Overwrite) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -863,7 +590,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(plans.size === 4) val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -975,7 +702,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -997,7 +724,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil) @@ -1149,32 +876,7 @@ class GlutenClickHouseDeltaParquetWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`"), compareResult = false) { df => val result = df.collect() assert(result.length === 2) @@ -1192,7 +894,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(scanExec.size === 1) val parquetScan = scanExec.head - assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.nodeName.startsWith("ScanTransformer parquet")) assert(parquetScan.metrics("numFiles").value === 200) val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -1215,35 +917,10 @@ class GlutenClickHouseDeltaParquetWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | delta.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"delta.`$dataPath`")) { _ => {} } } - testSparkVersionLE33("test parquet optimize basic") { + test("test parquet optimize basic") { withSQLConf("spark.databricks.delta.optimize.maxFileSize" -> "20000000") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize; @@ -1278,7 +955,7 @@ class GlutenClickHouseDeltaParquetWriteSuite } } - testSparkVersionLE33("test parquet optimize partitioned by one low card column") { + test("test parquet optimize partitioned by one low card column") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p2; |""".stripMargin) @@ -1317,7 +994,7 @@ class GlutenClickHouseDeltaParquetWriteSuite assert(ret2.apply(0).get(0) === 600572) } - testSparkVersionLE33("test parquet optimize parallel delete") { + test("test parquet optimize parallel delete") { withSQLConf("spark.databricks.delta.vacuum.parallelDelete.enabled" -> "true") { spark.sql(s""" |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p4; diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala similarity index 99% rename from backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala rename to backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala index 88a34a786a8c..7c444d572b69 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseFileFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseExcelFormatSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.spark.SparkConf @@ -46,7 +47,7 @@ case class AllDataTypesWithNonPrimitiveType( // data: (Seq[Int], (Int, String)) ) -class GlutenClickHouseFileFormatSuite +class GlutenClickHouseExcelFormatSuite extends GlutenClickHouseTPCHAbstractSuite with AdaptiveSparkPlanHelper { import testImplicits._ @@ -76,7 +77,7 @@ class GlutenClickHouseFileFormatSuite // in this case, FakeRowAdaptor does R2C test("parquet native writer writing a in memory DF") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test" val format = "parquet" @@ -99,7 +100,7 @@ class GlutenClickHouseFileFormatSuite // in this case, FakeRowAdaptor only wrap&transfer test("parquet native writer writing a DF from file") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test" val format = "parquet" @@ -123,7 +124,7 @@ class GlutenClickHouseFileFormatSuite // in this case, FakeRowAdaptor only wrap&transfer test("parquet native writer writing a DF from an aggregate") { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { val filePath = basePath + "/native_parquet_test_agg" val format = "parquet" @@ -273,7 +274,8 @@ class GlutenClickHouseFileFormatSuite StructField.apply("boolean_field", BooleanType, nullable = true), StructField.apply("decimal_field", DecimalType.apply(10, 2), nullable = true), StructField.apply("date_field", DateType, nullable = true), - StructField.apply("timestamp_field", TimestampType, nullable = true) + StructField.apply("timestamp_field", TimestampType, nullable = true), + StructField.apply("boolean_field2", BooleanType, nullable = true) )) val options = new util.HashMap[String, String]() @@ -1479,7 +1481,7 @@ class GlutenClickHouseFileFormatSuite | from $format.`$tablePath` | where long_field > 30 |""".stripMargin - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { testFileFormatBase(tablePath, format, sql, df => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala index 0205bae2d612..fbaebdecaaf6 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHAbstractSuite.scala @@ -621,4 +621,42 @@ abstract class GlutenClickHouseTPCHAbstractSuite checkDataFrame(noFallBack, customCheck, df) } + def q1(tableName: String): String = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | $tableName + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + + def q6(tableName: String): String = + s""" + |SELECT + | sum(l_extendedprice * l_discount) AS revenue + |FROM + | $tableName + |WHERE + | l_shipdate >= date'1994-01-01' + | AND l_shipdate < date'1994-01-01' + interval 1 year + | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 + | AND l_quantity < 24 + |""".stripMargin } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala index 6a09bf494217..90e09e75f1ff 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -40,6 +40,7 @@ class GlutenClickHouseTPCHBucketSuite override protected val queriesResults: String = rootPath + "bucket-queries-output" override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -47,6 +48,7 @@ class GlutenClickHouseTPCHBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } override protected val createNullableTables = true diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala index a56f45d1ba3d..65a01dea3073 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseTPCHSuite.scala @@ -47,7 +47,7 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { } assert(scanExec.size == 1) - assert(scanExec(0).nodeName.startsWith("Scan mergetree")) + assert(scanExec(0).nodeName.startsWith("ScanTransformer mergetree")) val sortExec = df.queryExecution.executedPlan.collect { case sortExec: SortExecTransformer => sortExec @@ -563,5 +563,32 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { compareResultsAgainstVanillaSpark(sql, true, { _ => }) spark.sql("drop table t1") } + + test("GLUTEN-7780 fix split diff") { + val sql = "select split(concat('a|b|c', cast(id as string)), '\\|')" + + ", split(concat('a|b|c', cast(id as string)), '\\\\|')" + + ", split(concat('a|b|c', cast(id as string)), '|') from range(10)" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } + test("GLUTEN-8142 duplicated columns in group by") { + sql("create table test_8142 (day string, rtime int, uid string, owner string) using parquet") + sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')") + sql("insert into test_8142 values ('2024-09-01', 123, 'user1', 'owner1')") + sql("insert into test_8142 values ('2024-09-02', 567, 'user2', 'owner2')") + compareResultsAgainstVanillaSpark( + """ + |select days, rtime, uid, owner, day1 + |from ( + | select day1 as days, rtime, uid, owner, day1 + | from ( + | select distinct coalesce(day, "today") as day1, rtime, uid, owner + | from test_8142 where day = '2024-09-01' + | )) group by days, rtime, uid, owner, day1 + |""".stripMargin, + true, + { _ => } + ) + sql("drop table test_8142") + } } // scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala index 0bd19dd97172..d7b277edfac5 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseWholeStageTransformerSuite.scala @@ -17,6 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.clickhouse.RuntimeConfig import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} @@ -41,20 +42,20 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu } val SPARK_DIR_NAME: String = sparkVersion.replace(".", "-") - val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME/" + val S3_METADATA_PATH = s"/tmp/metadata/s3/$SPARK_DIR_NAME" val S3_CACHE_PATH = s"/tmp/s3_cache/$SPARK_DIR_NAME/" val S3_ENDPOINT = "s3://127.0.0.1:9000/" val MINIO_ENDPOINT: String = S3_ENDPOINT.replace("s3", "http") val BUCKET_NAME: String = SPARK_DIR_NAME val WHOLE_PATH: String = MINIO_ENDPOINT + BUCKET_NAME + "/" - val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME/" + val HDFS_METADATA_PATH = s"/tmp/metadata/hdfs/$SPARK_DIR_NAME" val HDFS_CACHE_PATH = s"/tmp/hdfs_cache/$SPARK_DIR_NAME/" val HDFS_URL_ENDPOINT = "hdfs://127.0.0.1:8020" val HDFS_URL = s"$HDFS_URL_ENDPOINT/$SPARK_DIR_NAME" - val S3_ACCESS_KEY = "BypTYzcXOlfr03FFIvt4" - val S3_SECRET_KEY = "K9MDaGItPSaphorZM8t4hXf30gHF9dBWi6L2dK5E" + val S3_ACCESS_KEY = "minioadmin" + val S3_SECRET_KEY = "minioadmin" val CH_DEFAULT_STORAGE_DIR = "/data" @@ -82,7 +83,8 @@ class GlutenClickHouseWholeStageTransformerSuite extends WholeStageTransformerSu .set("spark.gluten.sql.enable.native.validation", "false") .set("spark.sql.warehouse.dir", warehouse) .setCHConfig("user_defined_path", "/tmp/user_defined") - .setCHConfig("path", UTSystemParameters.diskOutputDataPath) + .set(RuntimeConfig.PATH.key, UTSystemParameters.diskOutputDataPath) + .set(RuntimeConfig.TMP_PATH.key, s"/tmp/libch/$SPARK_DIR_NAME") if (UTSystemParameters.testMergeTreeOnObjectStorage) { conf .set("spark.hadoop.fs.s3a.access.key", S3_ACCESS_KEY) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala index 08342e588730..dbe8852290aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenFunctionValidateSuite.scala @@ -855,4 +855,9 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS compareResultsAgainstVanillaSpark(sql, true, { _ => }) } } + + test("GLUTEN-7796 cast bool to string") { + val sql = "select cast(id % 2 = 1 as string) from range(10)" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala index 2cfe935ef7e0..2437ffd03550 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/compatibility/GlutenClickhouseFunctionSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.compatibility import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.GlutenClickHouseTPCHAbstractSuite +import org.apache.gluten.execution.{GlutenClickHouseTPCHAbstractSuite, ProjectExecTransformer} import org.apache.gluten.utils.UTSystemParameters import org.apache.spark.SparkConf @@ -64,7 +64,7 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { test("test uuid - write and read") { withSQLConf( - ("spark.gluten.sql.native.writer.enabled", "true"), + (GlutenConfig.NATIVE_WRITER_ENABLED.key, "true"), (GlutenConfig.GLUTEN_ENABLED.key, "true")) { withTable("uuid_test") { spark.sql("create table if not exists uuid_test (id string) using parquet") @@ -269,6 +269,22 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { } } + test("GLUTEN-7594: cast const map to string") { + withSQLConf( + ( + "spark.sql.optimizer.excludedRules", + "org.apache.spark.sql.catalyst.optimizer.ConstantFolding," + + "org.apache.spark.sql.catalyst.optimizer.NullPropagation")) { + runQueryAndCompare( + """ + |select cast(map(1,'2') as string) + |""".stripMargin, + true, + false + )(checkGlutenOperatorMatch[ProjectExecTransformer]) + } + } + test("GLUTEN-7550 get_json_object in IN") { withTable("test_7550") { sql("create table test_7550(a string) using parquet") @@ -385,4 +401,18 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { } } + test("GLUTEN-8148: Fix corr with NaN") { + withTable("corr_nan") { + sql("create table if not exists corr_nan (x double, y double) using parquet") + sql("insert into corr_nan values(0,1)") + compareResultsAgainstVanillaSpark( + """ + |select corr(x,y), corr(y,x) from corr_nan + """.stripMargin, + true, + { _ => } + ) + } + } + } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala index 4e2b5ad63e0a..14d6ff53649f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseHiveTableSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.hive import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.{GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} +import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseWholeStageTransformerSuite, ProjectExecTransformer, TransformSupport} import org.apache.gluten.test.AllDataTypesWithComplexType import org.apache.gluten.utils.UTSystemParameters @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, StructType} import org.apache.hadoop.fs.Path @@ -542,6 +543,37 @@ class GlutenClickHouseHiveTableSuite ) } + test("GLUTEN-7700: test hive table with partition values contain space") { + val tbl = "test_7700" + val create_table_sql = + s""" + |create table if not exists $tbl ( + | id int + |) partitioned by (itime string) + |stored as orc; + |""".stripMargin + val insert_sql = + s""" + |insert overwrite table $tbl partition (itime = '2024-10-24 10:02:04') + |select id from range(3) + |""".stripMargin + val select_sql = + s""" + |select * from $tbl + |""".stripMargin + val drop_sql = s"drop table if exists $tbl" + + spark.sql(create_table_sql) + spark.sql(insert_sql) + + compareResultsAgainstVanillaSpark( + select_sql, + compareResult = true, + df => assert(df.count() == 3) + ) + spark.sql(drop_sql) + } + test("test hive compressed txt table") { withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> "11") { Seq("DefaultCodec", "BZip2Codec").foreach { @@ -1419,4 +1451,191 @@ class GlutenClickHouseHiveTableSuite spark.sql("DROP TABLE test_tbl_7054") } + test("Nested column pruning for Project(Filter(Generate))") { + spark.sql("drop table if exists aj") + spark.sql( + """ + |CREATE TABLE if not exists aj ( + | country STRING, + | event STRUCT, event_id:STRING, event_info:MAP> + |) + |USING orc + """.stripMargin) + + spark.sql(""" + |INSERT INTO aj VALUES + | ('USA', named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net', + | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1', + | 'event_info', map('tab_type', '5', 'action', '13'))), + | ('Canada', named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net', + | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2', + | 'event_info', map('tab_type', '4', 'action', '12'))) + """.stripMargin) + + val sql = """ + | SELECT * FROM ( + | SELECT + | game_name, + | CASE WHEN + | event.event_info['tab_type'] IN (5) THEN '1' ELSE '0' END AS entrance + | FROM aj + | LATERAL VIEW explode(split(nvl(event.event_info['game_name'],'0'),',')) + | as game_name + | WHERE event.event_info['action'] IN (13) + |) WHERE game_name = 'xxx' + """.stripMargin + + compareResultsAgainstVanillaSpark( + sql, + compareResult = true, + df => { + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + val fieldType = scan.schema.fields.head.dataType.asInstanceOf[StructType] + assert(fieldType.size == 1) + } + ) + + spark.sql("drop table if exists aj") + } + + test("Nested column pruning for Project(Filter(Generate)) on generator") { + def assertFieldSizeAfterPruning(sql: String, expectSize: Int): Unit = { + compareResultsAgainstVanillaSpark( + sql, + compareResult = true, + df => { + val scan = df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val fieldType = + scan.schema.fields.head.dataType + .asInstanceOf[ArrayType] + .elementType + .asInstanceOf[StructType] + assert(fieldType.size == expectSize) + } + ) + } + + spark.sql("drop table if exists ajog") + spark.sql( + """ + |CREATE TABLE if not exists ajog ( + | country STRING, + | events ARRAY, event_id:STRING, event_info:MAP>> + |) + |USING orc + """.stripMargin) + + spark.sql(""" + |INSERT INTO ajog VALUES + | ('USA', array(named_struct('time', 1622547800, 'lng', -122, 'lat', 37, 'net', + | 'wifi', 'log_extra', map('key1', 'value1'), 'event_id', 'event1', + | 'event_info', map('tab_type', '5', 'action', '13')))), + | ('Canada', array(named_struct('time', 1622547801, 'lng', -79, 'lat', 43, 'net', + | '4g', 'log_extra', map('key2', 'value2'), 'event_id', 'event2', + | 'event_info', map('tab_type', '4', 'action', '12')))) + """.stripMargin) + + // Test nested column pruning on generator with single field extracted + val sql1 = """ + |select + |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance + |from ajog + |lateral view explode(events) as event + |where event.event_info['action'] in (13) + """.stripMargin + assertFieldSizeAfterPruning(sql1, 1) + + // Test nested column pruning on generator with multiple field extracted, + // which resolves SPARK-34956 + val sql2 = """ + |select event.event_id, + |case when event.event_info['tab_type'] in (5) then '1' else '0' end as entrance + |from ajog + |lateral view explode(events) as event + |where event.event_info['action'] in (13) + """.stripMargin + assertFieldSizeAfterPruning(sql2, 2) + + // Test nested column pruning with two adjacent generate operator + val sql3 = """ + |SELECT + |abflag, + |event.event_info, + |event.log_extra + |FROM + |ajog + |LATERAL VIEW EXPLODE(events) AS event + |LATERAL VIEW EXPLODE(split(event.log_extra['key1'], ',')) AS abflag + |WHERE + |event.event_id = 'event1' + |AND event.event_info['tab_type'] IS NOT NULL + |AND event.event_info['tab_type'] != '' + |AND event.log_extra['key1'] = 'value1' + |LIMIT 100; + """.stripMargin + assertFieldSizeAfterPruning(sql3, 3) + + spark.sql("drop table if exists ajog") + } + + test("test hive table scan nested column pruning") { + val json_table_name = "test_tbl_7267_json" + val pq_table_name = "test_tbl_7267_pq" + val create_table_sql = + s""" + | create table if not exists %s( + | id bigint, + | d1 STRUCT>>, + | d2 STRUCT>>, + | day string, + | hour string + | ) partitioned by(day, hour) + |""".stripMargin + val create_table_json = create_table_sql.format(json_table_name) + + s""" + | ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' + | STORED AS INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + |""".stripMargin + val create_table_pq = create_table_sql.format(pq_table_name) + " Stored as PARQUET" + val insert_sql = + """ + | insert into %s values(1, + | named_struct('c', 'c123', 'd', array(named_struct('x', 'x123', 'y', 'y123'))), + | named_struct('c', 'c124', 'd', map('m124', named_struct('x', 'x124', 'y', 'y124'))), + | '2024-09-26', '12' + | ) + |""".stripMargin + val select_sql = + "select id, d1.c, d1.d[0].x, d2.d['m124'].y from %s where day = '2024-09-26' and hour = '12'" + val table_names = Array.apply(json_table_name, pq_table_name) + val create_table_sqls = Array.apply(create_table_json, create_table_pq) + for (i <- table_names.indices) { + val table_name = table_names(i) + val create_table = create_table_sqls(i) + spark.sql(create_table) + spark.sql(insert_sql.format(table_name)) + withSQLConf(("spark.sql.hive.convertMetastoreParquet" -> "false")) { + compareResultsAgainstVanillaSpark( + select_sql.format(table_name), + compareResult = true, + df => { + val scan = collect(df.queryExecution.executedPlan) { + case l: HiveTableScanExecTransformer => l + } + assert(scan.size == 1) + } + ) + } + spark.sql("drop table if exists %s".format(table_name)) + } + } + } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala index 53aef16d143e..16ed302a02f4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseNativeWriteTableSuite.scala @@ -553,7 +553,7 @@ class GlutenClickHouseNativeWriteTableSuite // spark write does not support bucketed table // https://issues.apache.org/jira/browse/SPARK-19256 val table_name = table_name_template.format(format) - writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, fields_.keys.toSeq) { fields => spark .table("origin_table") @@ -589,8 +589,9 @@ class GlutenClickHouseNativeWriteTableSuite ("byte_field", "byte"), ("boolean_field", "boolean"), ("decimal_field", "decimal(23,12)"), - ("date_field", "date"), - ("timestamp_field", "timestamp") + ("date_field", "date") + // ("timestamp_field", "timestamp") + // FIXME https://github.com/apache/incubator-gluten/issues/8053 ) val origin_table = "origin_table" withSource(genTestData(), origin_table) { @@ -598,7 +599,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) val testFields = fields.keys.toSeq - writeAndCheckRead(origin_table, table_name, testFields, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, testFields) { fields => spark .table(origin_table) @@ -610,7 +611,7 @@ class GlutenClickHouseNativeWriteTableSuite } val table_name_vanilla = table_name_vanilla_template.format(format) spark.sql(s"drop table IF EXISTS $table_name_vanilla") - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { withNativeWriteCheck(checkNative = false) { spark .table("origin_table") @@ -658,7 +659,7 @@ class GlutenClickHouseNativeWriteTableSuite nativeWrite { format => val table_name = table_name_template.format(format) - writeAndCheckRead(origin_table, table_name, fields.keys.toSeq, isSparkVersionLE("3.3")) { + writeAndCheckRead(origin_table, table_name, fields.keys.toSeq) { fields => spark .table("origin_table") @@ -672,7 +673,7 @@ class GlutenClickHouseNativeWriteTableSuite val table_name_vanilla = table_name_vanilla_template.format(format) spark.sql(s"drop table IF EXISTS $table_name_vanilla") - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { withNativeWriteCheck(checkNative = false) { spark .table("origin_table") @@ -762,7 +763,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + withNativeWriteCheck(checkNative = true) { spark .range(10000000) .selectExpr("id", "cast('2020-01-01' as date) as p") @@ -798,7 +799,7 @@ class GlutenClickHouseNativeWriteTableSuite format => val table_name = table_name_template.format(format) spark.sql(s"drop table IF EXISTS $table_name") - withNativeWriteCheck(checkNative = isSparkVersionLE("3.3")) { + withNativeWriteCheck(checkNative = true) { spark .range(30000) .selectExpr("id", "cast(null as string) as p") @@ -903,7 +904,7 @@ class GlutenClickHouseNativeWriteTableSuite } test("GLUTEN-2584: fix native write and read mismatch about complex types") { - def table(format: String): String = s"t_$format" + def table(format: String): String = s"t_2584_$format" def create(format: String, table_name: Option[String] = None): String = s"""CREATE TABLE ${table_name.getOrElse(table(format))}( | id INT, @@ -934,4 +935,63 @@ class GlutenClickHouseNativeWriteTableSuite } ) } + + test( + "GLUTEN-8021/8022/8032: fix orc read/write mismatch and parquet" + + "read exception when written complex column contains null") { + def table(format: String): String = s"t_8021_$format" + def create(format: String, table_name: Option[String] = None): String = + s"""CREATE TABLE ${table_name.getOrElse(table(format))}( + |id int, + |x int, + |y int, + |mp map, + |arr array, + |tup struct, + |arr_mp array>, + |mp_arr map>, + |tup_arr struct>, + |tup_map struct> + |) stored as $format""".stripMargin + def insert(format: String, table_name: Option[String] = None): String = + s"""INSERT OVERWRITE TABLE ${table_name.getOrElse(table(format))} + |SELECT + | id, x, y, + | str_to_map(concat('x:', x, ',y:', y)) AS mp, + | IF(id % 4 = 0, NULL, array(x, y)) AS arr, + | IF(id % 4 = 1, NULL, struct(x, y)) AS tup, + | IF(id % 4 = 2, NULL, array(str_to_map(concat('x:', x, ',y:', y)))) AS arr_mp, + | IF(id % 4 = 3, NULL, map('x', array(x), 'y', array(y))) AS mp_arr, + | IF(id % 4 = 0, NULL, named_struct('a', array(x, y))) AS tup_arr, + | IF(id % 4 = 1, NULL, named_struct('m', + | str_to_map(concat('x:', x, ',y:', y)))) AS tup_map + |FROM ( + | SELECT + | id, + | IF(id % 3 = 1, NULL, id + 1) AS x, + | IF(id % 3 = 1, NULL, id + 2) AS y + | FROM range(100) + |) AS data_source;""".stripMargin + + // TODO fix it in spark3.5 + if (!isSparkVersionGE("3.5")) { + nativeWrite2( + format => (table(format), create(format), insert(format)), + (table_name, format) => { + val vanilla_table = s"${table_name}_v" + val vanilla_create = create(format, Some(vanilla_table)) + vanillaWrite { + withDestinationTable(vanilla_table, Option(vanilla_create)) { + checkInsertQuery(insert(format, Some(vanilla_table)), checkNative = false) + } + } + val rowsFromOriginTable = + spark.sql(s"select * from $vanilla_table").collect() + val dfFromWriteTable = + spark.sql(s"select * from $table_name") + checkAnswer(dfFromWriteTable, rowsFromOriginTable) + } + ) + } + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala index 503c9bd886ed..5c5f93c62c6b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/hive/GlutenClickHouseTableAfterRestart.scala @@ -107,40 +107,14 @@ class GlutenClickHouseTableAfterRestart | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - // before restart, check if cache works { - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) val oldMissingCount1 = ClickhouseSnapshot.deltaScanCache.stats().missCount() val oldMissingCount2 = ClickhouseSnapshot.addFileToAddMTPCache.stats().missCount() // for this run, missing count should not increase - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) val stats1 = ClickhouseSnapshot.deltaScanCache.stats() assertResult(oldMissingCount1)(stats1.missCount()) val stats2 = ClickhouseSnapshot.addFileToAddMTPCache.stats() @@ -152,7 +126,7 @@ class GlutenClickHouseTableAfterRestart restartSpark() - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_mergetree"))(_ => {}) // after restart, additionally check stats of delta scan cache val stats1 = ClickhouseSnapshot.deltaScanCache.stats() diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala index a85a9094d38f..a01c708adabe 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeCacheDataSuite.scala @@ -134,7 +134,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) @@ -178,7 +178,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -238,7 +238,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) eventually(timeout(60.seconds), interval(2.seconds)) { assertResult(22)(metaPath.list().length) @@ -286,7 +286,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -346,7 +346,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) @@ -389,7 +389,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -439,7 +439,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite val dataPath = new File(HDFS_CACHE_PATH) val initial_cache_files = countFiles(dataPath) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") val res1 = spark.sql(s"cache data select * from lineitem_mergetree_hdfs").collect() assertResult(true)(res1(0).getBoolean(0)) assertResult(1)(metaPath.list().length) @@ -478,7 +478,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -539,7 +539,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite | aaa='ccc')""".stripMargin) .collect() assertResult(true)(res(0).getBoolean(0)) - val metaPath = new File(HDFS_METADATA_PATH + s"$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") + val metaPath = new File(s"$HDFS_METADATA_PATH/$SPARK_DIR_NAME/test/lineitem_mergetree_hdfs") assertResult(true)(metaPath.exists() && metaPath.isDirectory) assertResult(22)(metaPath.list().length) assert(countFiles(dataPath) > initial_cache_files) @@ -582,7 +582,7 @@ class GlutenClickHouseMergeTreeCacheDataSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala index 62b9ee3bcb31..76b3ba9f3724 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreePathBasedWriteSuite.scala @@ -114,32 +114,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Overwrite) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -148,7 +123,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(4)(plans.size) val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -205,32 +180,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .option("clickhouse.lowCardKey", "l_returnflag,l_linestatus") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val plans = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -239,7 +189,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(4)(plans.size) val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -383,7 +333,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -416,7 +366,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -575,32 +525,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -608,7 +533,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -672,32 +597,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -715,7 +615,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(3744)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -771,32 +671,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -804,7 +679,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -875,32 +750,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -908,7 +758,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -935,32 +785,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { _ => {} } } @@ -978,32 +803,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { _ => {} } val directory = new File(dataPath) // find a folder whose name is like 48b70783-b3b8-4bf8-9c52-5261aead8e3e_0_006 val partDir = directory.listFiles().filter(f => f.getName.length > 20).head @@ -1057,19 +857,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite .mode(SaveMode.Append) .save(dataPath) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1077,7 +865,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1218,32 +1006,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite test("GLUTEN-5219: Fix the table metadata sync issue for the CH backend") { def checkQueryResult(tableName: String): Unit = { - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$tableName` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$tableName`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1251,7 +1014,7 @@ class GlutenClickHouseMergeTreePathBasedWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala index ca1c9eef2a77..b147173d255b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSSuite.scala @@ -111,32 +111,8 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -144,7 +120,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -196,32 +172,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -229,7 +180,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -358,32 +309,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_hdfs"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -401,7 +327,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -471,32 +397,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -504,7 +405,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -552,32 +453,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite .option("clickhouse.storage_policy", "__hdfs_main") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -585,7 +461,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala index 26c6dd852b89..2540186c9984 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite.scala @@ -111,32 +111,8 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(HDFS_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -144,7 +120,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -196,32 +172,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -229,7 +180,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -358,32 +309,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_hdfs"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -401,7 +327,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -471,32 +397,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_hdfs - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_hdfs")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -504,7 +405,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -552,32 +453,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite .option("clickhouse.storage_policy", "__hdfs_main_rocksdb") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -585,7 +461,7 @@ class GlutenClickHouseMergeTreeWriteOnHDFSWithRocksDBMetaSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala index af67b01f49f0..8e698adb4b51 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteOnS3Suite.scala @@ -125,32 +125,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) FileUtils.deleteDirectory(new File(S3_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -158,7 +134,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -196,9 +172,9 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite val objectName = obj.get().objectName() if (objectName.contains("metadata.gluten")) { metadataGlutenExist = true - } else if (objectName.contains("meta.bin")) { + } else if (objectName.contains("part_meta.gluten")) { metadataBinExist = true - } else if (objectName.contains("data.bin")) { + } else if (objectName.contains("part_data.gluten")) { dataBinExist = true } else if (objectName.contains("_commits")) { // Spark 35 has _commits directory @@ -256,32 +232,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -289,7 +240,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -418,32 +369,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | where l_returnflag = 'A' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition_s3"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -461,7 +387,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(6)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -532,32 +458,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket_s3 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket_s3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -565,7 +466,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -613,32 +514,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite .option("clickhouse.storage_policy", "__s3_main") .save(dataPath) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | clickhouse.`$dataPath` - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(s"clickhouse.`$dataPath`")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -646,7 +522,7 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -743,21 +619,8 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite FileUtils.forceDelete(new File(S3_METADATA_PATH)) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | $tableName - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - withSQLConf(runtimeSettings("enabled_driver_filter_mergetree_index") -> "true") { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6(tableName)) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -765,13 +628,13 @@ class GlutenClickHouseMergeTreeWriteOnS3Suite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala index c1210c5fbaca..cc577609656b 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution.mergetree +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution._ import org.apache.gluten.utils.Arm @@ -55,7 +56,7 @@ class GlutenClickHouseMergeTreeWriteSuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.sql.adaptive.enabled", "true") .set("spark.sql.files.maxPartitionBytes", "20000000") - .set("spark.gluten.sql.native.writer.enabled", "true") + .set(GlutenConfig.NATIVE_WRITER_ENABLED.key, "true") .setCHSettings("min_insert_block_size_rows", 100000) .setCHSettings("mergetree.merge_after_insert", false) .setCHSettings("input_format_parquet_max_block_size", 8192) @@ -66,197 +67,178 @@ class GlutenClickHouseMergeTreeWriteSuite } test("test mergetree table write") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree; + |""".stripMargin) - // write.format.default = mergetree - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |TBLPROPERTIES (write.format.default = 'mergetree') - |LOCATION '$basePath/lineitem_mergetree' - |""".stripMargin) + // write.format.default = mergetree + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |TBLPROPERTIES (write.format.default = 'mergetree') + |LOCATION '$basePath/lineitem_mergetree' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree + | select * from lineitem + |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { - df => - val plans = collect(df.queryExecution.executedPlan) { - case f: FileSourceScanExecTransformer => f - case w: WholeStageTransformer => w - } - assertResult(4)(plans.size) + runTPCHQueryBySQL(1, q1("lineitem_mergetree")) { + df => + val plans = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + case w: WholeStageTransformer => w + } + assertResult(4)(plans.size) - val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + val mergetreeScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) - val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).primaryKeyOption.isEmpty) - assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.isEmpty) - val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) - assertResult(6)(addFiles.size) - assertResult(600572)(addFiles.map(_.rows).sum) + val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).bucketOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).orderByKeyOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).primaryKeyOption.isEmpty) + assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).partitionColumns.isEmpty) + val addFiles = + fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) + assertResult(6)(addFiles.size) + assertResult(600572)(addFiles.map(_.rows).sum) - // GLUTEN-5060: check the unnecessary FilterExec - val wholeStageTransformer = plans(2).asInstanceOf[WholeStageTransformer] - val planNodeJson = wholeStageTransformer.substraitPlanJson - assert( - !planNodeJson - .replaceAll("\n", "") - .replaceAll(" ", "") - .contains("\"input\":{\"filter\":{")) + // GLUTEN-5060: check the unnecessary FilterExec + val wholeStageTransformer = plans(2).asInstanceOf[WholeStageTransformer] + val planNodeJson = wholeStageTransformer.substraitPlanJson + assert( + !planNodeJson + .replaceAll("\n", "") + .replaceAll(" ", "") + .contains("\"input\":{\"filter\":{")) + } } - } test("test mergetree insert overwrite") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite + | select * from lineitem + |""".stripMargin) - spark.sql(s""" - | insert overwrite table lineitem_mergetree_insertoverwrite - | select * from lineitem where mod(l_orderkey,2) = 1 - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite - | - |""".stripMargin - assertResult(300001)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql(s""" + | insert overwrite table lineitem_mergetree_insertoverwrite + | select * from lineitem where mod(l_orderkey,2) = 1 + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite + | + |""".stripMargin + assertResult(300001)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, static") { - spark.sql(s""" - |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; - |""".stripMargin) + withSQLConf((CHConf.ENABLE_ONEPIPELINE_MERGETREE_WRITE.key, spark35.toString)) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_insertoverwrite2; + |""".stripMargin) - spark.sql(s""" - |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 - |( - | l_orderkey bigint, - | l_partkey bigint, - | l_suppkey bigint, - | l_linenumber bigint, - | l_quantity double, - | l_extendedprice double, - | l_discount double, - | l_tax double, - | l_returnflag string, - | l_linestatus string, - | l_shipdate date, - | l_commitdate date, - | l_receiptdate date, - | l_shipinstruct string, - | l_shipmode string, - | l_comment string - |) - |USING clickhouse - |PARTITIONED BY (l_shipdate) - |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' - |""".stripMargin) + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_insertoverwrite2 + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_mergetree_insertoverwrite2' + |""".stripMargin) - spark.sql(s""" - | insert into table lineitem_mergetree_insertoverwrite2 - | select * from lineitem - |""".stripMargin) + spark.sql(s""" + | insert into table lineitem_mergetree_insertoverwrite2 + | select * from lineitem + |""".stripMargin) - spark.sql( - s""" - | insert overwrite table lineitem_mergetree_insertoverwrite2 - | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' - |""".stripMargin) - val sql2 = - s""" - | select count(*) from lineitem_mergetree_insertoverwrite2 - | - |""".stripMargin - assertResult(2418)( - // total rows should remain unchanged - spark.sql(sql2).collect().apply(0).get(0) - ) + spark.sql( + s""" + | insert overwrite table lineitem_mergetree_insertoverwrite2 + | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_mergetree_insertoverwrite2 + | + |""".stripMargin + assertResult(2418)( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) + ) + } } test("test mergetree insert overwrite partitioned table with small table, dynamic") { @@ -567,32 +549,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_orderbykey - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_orderbykey")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -600,7 +557,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -725,32 +682,8 @@ class GlutenClickHouseMergeTreeWriteSuite | l_comment from lineitem | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_partition - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + + runTPCHQueryBySQL(1, q1("lineitem_mergetree_partition"), compareResult = false) { df => val result = df.collect() assertResult(4)(result.length) @@ -768,7 +701,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) assertResult(3745)(mergetreeScan.metrics("numFiles").value) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] @@ -843,32 +776,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_bucket - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_bucket")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -876,7 +784,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1049,32 +957,7 @@ class GlutenClickHouseMergeTreeWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_ctas1 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1("lineitem_mergetree_ctas1")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1082,7 +965,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1111,32 +994,7 @@ class GlutenClickHouseMergeTreeWriteSuite | as select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_ctas2 - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1("lineitem_mergetree_ctas2")) { _ => {} } } @@ -1175,32 +1033,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_mergetree_lowcard - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { _ => {} } + runTPCHQueryBySQL(1, q1("lineitem_mergetree_lowcard")) { _ => {} } val directory = new File(s"$basePath/lineitem_mergetree_lowcard") // find a folder whose name is like 48b70783-b3b8-4bf8-9c52-5261aead8e3e_0_006 val partDir = directory.listFiles().filter(f => f.getName.length > 20).head @@ -1275,19 +1108,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_orderbykey2 - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_orderbykey2")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1295,7 +1116,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1362,18 +1183,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_orderbykey3 - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_orderbykey3")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1381,7 +1191,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] assert(ClickHouseTableV2.getTable(fileIndex.deltaLog).clickhouseTableConfigs.nonEmpty) @@ -1505,7 +1315,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddMergeTreeParts]) @@ -1578,7 +1388,7 @@ class GlutenClickHouseMergeTreeWriteSuite | l_linestatus; | |""".stripMargin - runTPCHQueryBySQL(1, sqlStr) { + runTPCHQueryBySQL(1, q1(tableName)) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1586,7 +1396,7 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val fileIndex = mergetreeScan.relation.location.asInstanceOf[TahoeFileIndex] val addFiles = @@ -1771,23 +1581,10 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_pk_pruning_by_driver - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - Seq(("true", 2), ("false", 3)).foreach( conf => { withSQLConf(CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> conf._1) { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_pk_pruning_by_driver")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1795,13 +1592,13 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(conf._2)(plans.head.getSplitInfos(null).size) + assertResult(conf._2)(plans.head.getSplitInfos.size) } } }) @@ -1811,7 +1608,7 @@ class GlutenClickHouseMergeTreeWriteSuite CHConf.runtimeSettings("enabled_driver_filter_mergetree_index") -> "true", CHConf.prefixOf("files.per.partition.threshold") -> "10" ) { - runTPCHQueryBySQL(6, sqlStr) { + runTPCHQueryBySQL(6, q6("lineitem_mergetree_pk_pruning_by_driver")) { df => val scanExec = collect(df.queryExecution.executedPlan) { case f: FileSourceScanExecTransformer => f @@ -1819,13 +1616,13 @@ class GlutenClickHouseMergeTreeWriteSuite assertResult(1)(scanExec.size) val mergetreeScan = scanExec.head - assert(mergetreeScan.nodeName.startsWith("Scan mergetree")) + assert(mergetreeScan.nodeName.startsWith("ScanTransformer mergetree")) val plans = collect(df.queryExecution.executedPlan) { case scanExec: BasicScanExecTransformer => scanExec } assertResult(1)(plans.size) - assertResult(1)(plans.head.getSplitInfos(null).size) + assertResult(1)(plans.head.getSplitInfos.size) } } } @@ -1933,7 +1730,7 @@ class GlutenClickHouseMergeTreeWriteSuite case f: BasicScanExecTransformer => f } assertResult(2)(scanExec.size) - assertResult(conf._2)(scanExec(1).getSplitInfos(null).size) + assertResult(conf._2)(scanExec(1).getSplitInfos.size) } } }) @@ -1977,24 +1774,29 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | count(*) AS count_order - |FROM - | lineitem_mergetree_count_opti - |""".stripMargin - runSql(sqlStr)( - df => { - val result = df.collect() - assertResult(1)(result.length) - assertResult("600572")(result(0).getLong(0).toString) + Seq("true", "false").foreach { + skip => + withSQLConf("spark.databricks.delta.stats.skipping" -> skip.toString) { + val sqlStr = + s""" + |SELECT + | count(*) AS count_order + |FROM + | lineitem_mergetree_count_opti + |""".stripMargin + runSql(sqlStr)( + df => { + val result = df.collect() + assertResult(1)(result.length) + assertResult("600572")(result(0).getLong(0).toString) - // Spark 3.2 + Delta 2.0 does not support this feature - if (!spark32) { - assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + // Spark 3.2 + Delta 2.0 does not support this feature + if (!spark32) { + assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + } + }) } - }) + } } test("test mergetree with column case sensitive") { @@ -2033,19 +1835,7 @@ class GlutenClickHouseMergeTreeWriteSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | sum(l_extendedprice * l_discount) AS revenue - |FROM - | lineitem_mergetree_case_sensitive - |WHERE - | l_shipdate >= date'1994-01-01' - | AND l_shipdate < date'1994-01-01' + interval 1 year - | AND l_discount BETWEEN 0.06 - 0.01 AND 0.06 + 0.01 - | AND l_quantity < 24 - |""".stripMargin - runTPCHQueryBySQL(6, sqlStr) { _ => } + runTPCHQueryBySQL(6, q6("lineitem_mergetree_case_sensitive")) { _ => } } test("test mergetree with partition with whitespace") { @@ -2128,4 +1918,86 @@ class GlutenClickHouseMergeTreeWriteSuite } }) } + + test( + "GLUTEN-7812: Fix the query failed for the mergetree format " + + "when the 'spark.databricks.delta.stats.skipping' is off") { + // Spark 3.2 + Delta 2.0 doesn't not support this feature + if (!spark32) { + withSQLConf(("spark.databricks.delta.stats.skipping", "false")) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_mergetree_stats_skipping; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_mergetree_stats_skipping + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING clickhouse + |PARTITIONED BY (l_returnflag) + |TBLPROPERTIES (orderByKey='l_orderkey', + | primaryKey='l_orderkey') + |LOCATION '$basePath/lineitem_mergetree_stats_skipping' + |""".stripMargin) + + // dynamic partitions + spark.sql(s""" + | insert into table lineitem_mergetree_stats_skipping + | select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | o_orderpriority, + | count(*) AS order_count + |FROM + | orders + |WHERE + | o_orderdate >= date'1993-07-01' + | AND o_orderdate < date'1993-07-01' + interval 3 month + | AND EXISTS ( + | SELECT + | * + | FROM + | lineitem + | WHERE + | l_orderkey = o_orderkey + | AND l_commitdate < l_receiptdate) + |GROUP BY + | o_orderpriority + |ORDER BY + | o_orderpriority; + | + |""".stripMargin + runSql(sqlStr)( + df => { + val result = df.collect() + assertResult(5)(result.length) + assertResult("1-URGENT")(result(0).getString(0)) + assertResult(999)(result(0).getLong(1)) + assertResult("2-HIGH")(result(1).getString(0)) + assertResult(997)(result(1).getLong(1)) + assertResult("5-LOW")(result(4).getString(0)) + assertResult(1077)(result(4).getLong(1)) + }) + } + } + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala index 5f03a0d39f50..767585c4c41e 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/mergetree/GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite.scala @@ -86,31 +86,6 @@ class GlutenClickHouseMergeTreeWriteTaskNotSerializableSuite | select * from lineitem |""".stripMargin) - val sqlStr = - s""" - |SELECT - | l_returnflag, - | l_linestatus, - | sum(l_quantity) AS sum_qty, - | sum(l_extendedprice) AS sum_base_price, - | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, - | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, - | avg(l_quantity) AS avg_qty, - | avg(l_extendedprice) AS avg_price, - | avg(l_discount) AS avg_disc, - | count(*) AS count_order - |FROM - | lineitem_task_not_serializable - |WHERE - | l_shipdate <= date'1998-09-02' - interval 1 day - |GROUP BY - | l_returnflag, - | l_linestatus - |ORDER BY - | l_returnflag, - | l_linestatus; - | - |""".stripMargin - runTPCHQueryBySQL(1, sqlStr)(_ => {}) + runTPCHQueryBySQL(1, q1("lineitem_task_not_serializable"))(_ => {}) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala index cbf1caf44e7f..02445270d42c 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCDSMetricsSuite.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution.metrics import org.apache.gluten.execution.{ColumnarNativeIterator, GlutenClickHouseTPCDSAbstractSuite, WholeStageTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala index 932686433e5b..015da0dfae10 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/metrics/GlutenClickHouseTPCHMetricsSuite.scala @@ -17,11 +17,13 @@ package org.apache.gluten.execution.metrics import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.InputIteratorTransformer +import org.apache.spark.sql.execution.{ColumnarInputAdapter, InputIteratorTransformer} +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.task.TaskResources import scala.collection.JavaConverters._ @@ -422,4 +424,39 @@ class GlutenClickHouseTPCHMetricsSuite extends GlutenClickHouseTPCHAbstractSuite assert(nativeMetricsDataFinal.metricsDataList.get(2).getName.equals("kProject")) } } + + test("Metrics for input iterator of broadcast exchange") { + createTPCHNotNullTables() + val partTableRecords = spark.sql("select * from part").count() + + // Repartition to make sure we have multiple tasks executing the join. + spark + .sql("select * from lineitem") + .repartition(2) + .createOrReplaceTempView("lineitem") + + Seq("true", "false").foreach { + adaptiveEnabled => + withSQLConf("spark.sql.adaptive.enabled" -> adaptiveEnabled) { + val sqlStr = + """ + |select /*+ BROADCAST(part) */ * from part join lineitem + |on l_partkey = p_partkey + |""".stripMargin + + runQueryAndCompare(sqlStr) { + df => + val inputIterator = find(df.queryExecution.executedPlan) { + case InputIteratorTransformer(ColumnarInputAdapter(child)) => + child.isInstanceOf[BroadcastQueryStageExec] || child + .isInstanceOf[BroadcastExchangeLike] + case _ => false + } + assert(inputIterator.isDefined) + val metrics = inputIterator.get.metrics + assert(metrics("numOutputRows").value == partTableRecords) + } + } + } + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala index 389d617f10eb..52a72c44941f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetAQESuite.scala @@ -35,9 +35,6 @@ class GlutenClickHouseTPCDSParquetAQESuite .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.sql.adaptive.enabled", "true") .set("spark.memory.offHeap.size", "4g") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala index 3e965c67ea9a..e8fea04f2c6f 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite.scala @@ -20,6 +20,8 @@ import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.DynamicPruningExpression +import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.execution.{ColumnarSubqueryBroadcastExec, ReusedSubqueryExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec @@ -35,9 +37,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.sql.adaptive.enabled", "true") .set("spark.memory.offHeap.size", "4g") } @@ -265,4 +264,78 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleAQESuite } }) } + + test("GLUTEN-7971: Support using left side as the build table for the left anti/semi join") { + withSQLConf( + ("spark.sql.autoBroadcastJoinThreshold", "-1"), + ("spark.gluten.sql.columnar.backend.ch.convert.left.anti_semi.to.right", "true")) { + val sql1 = + s""" + |select + | cd_gender, + | cd_marital_status, + | cd_education_status, + | count(*) cnt1 + | from + | customer c,customer_address ca,customer_demographics + | where + | c.c_current_addr_sk = ca.ca_address_sk and + | ca_county in ('Walker County','Richland County','Gaines County','Douglas County') + | and cd_demo_sk = c.c_current_cdemo_sk and + | exists (select * + | from store_sales + | where c.c_customer_sk = ss_customer_sk) + | group by cd_gender, + | cd_marital_status, + | cd_education_status + | order by cd_gender, + | cd_marital_status, + | cd_education_status + | LIMIT 100 ; + |""".stripMargin + runQueryAndCompare(sql1)( + df => { + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftSemi => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + }) + + val sql2 = + s""" + |select + | cd_gender, + | cd_marital_status, + | cd_education_status, + | count(*) cnt1 + | from + | customer c,customer_address ca,customer_demographics + | where + | c.c_current_addr_sk = ca.ca_address_sk and + | ca_county in ('Walker County','Richland County','Gaines County','Douglas County') + | and cd_demo_sk = c.c_current_cdemo_sk and + | not exists (select * + | from store_sales + | where c.c_customer_sk = ss_customer_sk) + | group by cd_gender, + | cd_marital_status, + | cd_education_status + | order by cd_gender, + | cd_marital_status, + | cd_education_status + | LIMIT 100 ; + |""".stripMargin + runQueryAndCompare(sql2)( + df => { + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftAnti => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + }) + } + } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala index 4675de249c6d..24d8e9db607d 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetColumnarShuffleSuite.scala @@ -32,9 +32,6 @@ class GlutenClickHouseTPCDSParquetColumnarShuffleSuite extends GlutenClickHouseT .set("spark.io.compression.codec", "LZ4") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.memory.offHeap.size", "4g") // .set("spark.sql.planChangeLog.level", "error") } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala index 08f4522d9ce4..f0025cf30cad 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpcds/GlutenClickHouseTPCDSParquetSuite.scala @@ -30,17 +30,16 @@ class GlutenClickHouseTPCDSParquetSuite extends GlutenClickHouseTPCDSAbstractSui /** Run Gluten + ClickHouse Backend with SortShuffleManager */ override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "sort") .set("spark.io.compression.codec", "snappy") .set("spark.sql.shuffle.partitions", "5") .set("spark.sql.autoBroadcastJoinThreshold", "10MB") - // Currently, it can not support to read multiple partitioned file in one task. - // .set("spark.sql.files.maxPartitionBytes", "134217728") - // .set("spark.sql.files.openCostInBytes", "134217728") .set("spark.memory.offHeap.size", "4g") .set("spark.gluten.sql.validation.logLevel", "ERROR") .set("spark.gluten.sql.validation.printStackOnFailure", "true") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } executeTPCDSTest(false) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala index aa77f79bbac1..d79b25a7bd09 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseHDFSSuite.scala @@ -144,7 +144,7 @@ class GlutenClickHouseHDFSSuite test("GLUTEN-7542: Fix cache refresh") { withSQLConf("spark.sql.hive.manageFilesourcePartitions" -> "false") { - val filePath = s"$tablesPath/issue_7542/" + val filePath = s"$tablesPath/$SPARK_DIR_NAME/issue_7542/" val targetDirs = new Path(filePath) val fs = targetDirs.getFileSystem(spark.sessionState.newHadoopConf()) fs.mkdirs(targetDirs) diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index 885e50b0468b..b4186fee66aa 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -18,7 +18,6 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf @@ -52,6 +51,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") .setCHConfig("enable_streaming_aggregating", true) + .set(GlutenConfig.COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD.key, (1024 * 1024).toString) } override protected def createTPCHNotNullTables(): Unit = { @@ -116,6 +116,31 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite } } + // TODO: there is a bug when using timestamp type as the partition column + ignore("test timestamp as partition column") { + spark.sql(""" + |create table part_by_timestamp ( + | a int, + | b timestamp, + | c string, + | p timestamp + |) using parquet + |partitioned by (p); + |""".stripMargin) + + // Insert some test rows. + spark.sql(""" + |insert into table part_by_timestamp + |values + |(1, TIMESTAMP '2022-01-01 00:01:20', '2022-01-01 00:01:20', + |TIMESTAMP '2022-01-01 00:01:20'); + |""".stripMargin) + compareResultsAgainstVanillaSpark( + "select a, b, to_timestamp(c), p from part_by_timestamp", + compareResult = true, + customCheck = { _ => }) + } + test("TPCH Q2") { runTPCHQuery(2) { df => diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala index af72ba84a657..1c627140b694 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetAQESuite.scala @@ -20,6 +20,7 @@ import org.apache.gluten.execution._ import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.optimizer.BuildLeft +import org.apache.spark.sql.catalyst.plans.LeftSemi import org.apache.spark.sql.execution.{ReusedSubqueryExec, SubqueryExec} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper} @@ -213,6 +214,23 @@ class GlutenClickHouseTPCHParquetAQESuite runTPCHQuery(21) { df => } } + test( + "TPCH Q21 with GLUTEN-7971: Support using left side as the build table for the left anti/semi join") { + withSQLConf( + ("spark.sql.autoBroadcastJoinThreshold", "-1"), + ("spark.gluten.sql.columnar.backend.ch.convert.left.anti_semi.to.right", "true")) { + runTPCHQuery(21, compareResult = false) { + df => + assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) + val shuffledHashJoinExecs = collect(df.queryExecution.executedPlan) { + case h: CHShuffledHashJoinExecTransformer if h.joinType == LeftSemi => h + } + assertResult(1)(shuffledHashJoinExecs.size) + assertResult(BuildLeft)(shuffledHashJoinExecs(0).buildSide) + } + } + } + test("TPCH Q22") { runTPCHQuery(22) { df => diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala index 59b5834e64a1..7a927bf23a49 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -46,6 +46,7 @@ class GlutenClickHouseTPCHParquetBucketSuite protected val bucketTableDataPath: String = basePath + "/tpch-parquet-bucket" override protected def sparkConf: SparkConf = { + import org.apache.gluten.backendsapi.clickhouse.CHConf._ super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.io.compression.codec", "LZ4") @@ -53,6 +54,7 @@ class GlutenClickHouseTPCHParquetBucketSuite .set("spark.sql.autoBroadcastJoinThreshold", "-1") // for test bucket join .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") + .setCHConfig("enable_grace_aggregate_spill_test", "true") } override protected val createNullableTables = true diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala index 739b040dba1d..f82acdc415b2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/tpch/GlutenClickHouseTPCHSaltNullParquetSuite.scala @@ -19,7 +19,7 @@ package org.apache.gluten.execution.tpch import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.clickhouse.CHConf import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.DataFrame @@ -41,6 +41,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr override protected val tpchQueries: String = rootPath + "../../../../tools/gluten-it/common/src/main/resources/tpch-queries" override protected val queriesResults: String = rootPath + "queries-output" + val runtimeConfigPrefix = "spark.gluten.sql.columnar.backend.ch.runtime_config." override protected def sparkConf: SparkConf = { super.sparkConf @@ -2192,7 +2193,7 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr } } - test("GLUTEN-3135: Bug fix to_date") { + test("GLUTEN-3135/GLUTEN-7896: Bug fix to_date") { val create_table_sql = """ | create table test_tbl_3135(id bigint, data string) using parquet @@ -2209,13 +2210,27 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr |(7, '1970-01-01 00:00:00'), |(8, '2024-3-2'), |(9, '2024-03-2'), - |(10, '2024-03') + |(10, '2024-03'), + |(11, '2024-03-02 11:22:33') |""".stripMargin spark.sql(create_table_sql) spark.sql(insert_data_sql) val select_sql = "select id, to_date(data) from test_tbl_3135" compareResultsAgainstVanillaSpark(select_sql, true, { _ => }) + + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "corrected")) { + compareResultsAgainstVanillaSpark( + "select id, to_date('2024-03-2 11:22:33', 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } + withSQLConf(("spark.sql.legacy.timeParserPolicy" -> "legacy")) { + compareResultsAgainstVanillaSpark( + "select id, to_date(data, 'yyyy-MM-dd') from test_tbl_3135 where id = 11", + true, + { _ => }) + } spark.sql("drop table test_tbl_3135") } @@ -2458,6 +2473,56 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr runQueryAndCompare(sql, noFallBack = true)({ _ => }) } + test("GLUTEN-7979: fix different output schema array and array before union") { + val sql = + """ + |select + | a.multi_peer_user_id, + | max(a.user_id) as max_user_id, + | max(a.peer_user_id) as max_peer_user_id, + | max(a.is_match_line) as max_is_match_line + |from + |( + | select + | t1.user_id, + | t1.peer_user_id, + | t1.is_match_line, + | t1.pk_type, + | t1.pk_start_time, + | t1.pk_end_time, + | t1.multi_peer_user_id + | from + | ( + | select + | id as user_id, + | id as peer_user_id, + | id % 2 as is_match_line, + | id % 3 as pk_type, + | id as pk_start_time, + | id as pk_end_time, + | array() as multi_peer_user_id + | from range(10) + | + | union all + | + | select + | id as user_id, + | id as peer_user_id, + | id % 2 as is_match_line, + | id % 3 as pk_type, + | id as pk_start_time, + | id as pk_end_time, + | array('a', 'b', 'c') as multi_peer_user_id + | from range(10) + | ) t1 + | where t1.user_id > 0 and t1.peer_user_id > 0 + |) a + |group by + | a.multi_peer_user_id + |""".stripMargin + runQueryAndCompare(sql, noFallBack = true)({ _ => }) + } + test("GLUTEN-4190: crush on flattening a const null column") { val sql = """ @@ -3022,5 +3087,268 @@ class GlutenClickHouseTPCHSaltNullParquetSuite extends GlutenClickHouseTPCHAbstr compareResultsAgainstVanillaSpark(query_sql, true, { _ => }) spark.sql("drop table test_tbl_7220") } + + test("GLLUTEN-7647 lazy expand") { + def checkLazyExpand(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: ExpandExecTransformer if (e.child.isInstanceOf[HashAggregateExecBaseTransformer]) => + e + } + assert(expands.size == 1) + } + var sql = + """ + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select n_regionkey, n_nationkey, sum(n_regionkey), count(distinct n_name) + |from nation group by n_regionkey, n_nationkey with cube + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select * from( + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |) where n_regionkey != 0 + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select * from( + |select n_regionkey, n_nationkey, + |sum(n_regionkey), count(distinct n_name), max(n_regionkey), min(n_regionkey) + |from nation group by n_regionkey, n_nationkey with cube + |) where n_regionkey != 0 + |order by n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + + sql = """ + |select x, n_regionkey, n_nationkey, + |sum(n_regionkey), count(n_name), max(n_regionkey), min(n_regionkey) + |from (select '123' as x, * from nation) group by x, n_regionkey, n_nationkey with cube + |order by x, n_regionkey, n_nationkey + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + } + + test("GLUTEN-7647 lazy expand for avg and sum") { + val create_table_sql = + """ + |create table test_7647(x bigint, y bigint, z bigint, v decimal(10, 2)) using parquet + |""".stripMargin + spark.sql(create_table_sql) + val insert_data_sql = + """ + |insert into test_7647 values + |(1, 1, 1, 1.0), + |(2, 2, 2, 2.0), + |(3, 3, 3, 3.0), + |(2,2,1, 4.0) + |""".stripMargin + spark.sql(insert_data_sql) + + def checkLazyExpand(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: ExpandExecTransformer if (e.child.isInstanceOf[HashAggregateExecBaseTransformer]) => + e + } + assert(expands.size == 1) + } + + var sql = "select x, y, avg(z), sum(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + sql = + "select x, y, count(distinct z), avg(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + sql = + "select x, y, count(distinct z), sum(v) from test_7647 group by x, y with cube order by x, y" + compareResultsAgainstVanillaSpark(sql, true, checkLazyExpand) + spark.sql("drop table if exists test_7647") + } + + test("GLUTEN-7905 get topk of window by aggregate") { + withSQLConf( + (runtimeConfigPrefix + "enable_window_group_limit_to_aggregate", "true"), + (runtimeConfigPrefix + "window.aggregate_topk_high_cardinality_threshold", "2.0")) { + def checkWindowGroupLimit(df: DataFrame): Unit = { + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: CHAggregateGroupLimitExecTransformer => e + case wgl: CHWindowGroupLimitExecTransformer => wgl + } + assert(expands.size >= 1) + } + spark.sql("create table test_win_top (a string, b int, c int) using parquet") + spark.sql(""" + |insert into test_win_top values + |('a', 3, 3), ('a', 1, 5), ('a', 2, 2), ('a', null, null), ('a', null, 1), + |('b', 1, 1), ('b', 2, 1), + |('c', 2, 3) + |""".stripMargin) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, + |row_number() over (partition by a order by b desc nulls first, c nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b desc, c nulls last) as r + |from test_win_top + |)where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls first, c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b , c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + spark.sql("drop table if exists test_win_top") + } + + } + + test("GLUTEN-7905 get topk of window by window") { + withSQLConf( + (runtimeConfigPrefix + "enable_window_group_limit_to_aggregate", "true"), + (runtimeConfigPrefix + "window.aggregate_topk_high_cardinality_threshold", "0.0")) { + def checkWindowGroupLimit(df: DataFrame): Unit = { + // for spark 3.5, CHWindowGroupLimitExecTransformer is in used + val expands = collectWithSubqueries(df.queryExecution.executedPlan) { + case e: CHAggregateGroupLimitExecTransformer => e + case wgl: CHWindowGroupLimitExecTransformer => wgl + } + assert(expands.size >= 1) + } + spark.sql("drop table if exists test_win_top") + spark.sql("create table test_win_top (a string, b int, c int) using parquet") + spark.sql(""" + |insert into test_win_top values + |('a', 3, 3), ('a', 1, 5), ('a', 2, 2), ('a', null, null), ('a', null, 1), + |('b', 1, 1), ('b', 2, 1), + |('c', 2, 3) + |""".stripMargin) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, + |row_number() over (partition by a order by b desc nulls first, c nulls last) as r + |from test_win_top + |)where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b desc, c nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + | select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls first, c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b asc nulls last) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + compareResultsAgainstVanillaSpark( + """ + |select * from( + |select a, b, c, row_number() over (partition by a order by b , c) as r + |from test_win_top) + |where r <= 1 + |""".stripMargin, + true, + checkWindowGroupLimit + ) + spark.sql("drop table if exists test_win_top") + } + + } + + test("GLUTEN-7759: Fix bug of agg pre-project push down") { + val table_create_sql = + "create table test_tbl_7759(id bigint, name string, day string) using parquet" + val insert_data_sql = + "insert into test_tbl_7759 values(1, 'a123', '2024-11-01'),(2, 'a124', '2024-11-01')" + val query_sql = + """ + |select distinct day, name from( + |select '2024-11-01' as day + |,coalesce(name,'all') name + |,cnt + |from + |( + |select count(distinct id) as cnt, name + |from test_tbl_7759 + |group by name + |with cube + |)) limit 10 + |""".stripMargin + spark.sql(table_create_sql) + spark.sql(insert_data_sql) + compareResultsAgainstVanillaSpark(query_sql, true, { _ => }) + spark.sql("drop table test_tbl_7759") + } } // scalastyle:on line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala index 49e368c888e7..631461b6a2f4 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/gluten/NativeWriteChecker.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.gluten +import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.GlutenClickHouseWholeStageTransformerSuite import org.apache.spark.sql.{Dataset, Row} @@ -70,13 +71,13 @@ trait NativeWriteChecker } def nativeWrite(f: String => Unit): Unit = { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "true")) { formats.foreach(f(_)) } } def vanillaWrite(block: => Unit): Unit = { - withSQLConf(("spark.gluten.sql.native.writer.enabled", "false")) { + withSQLConf((GlutenConfig.NATIVE_WRITER_ENABLED.key, "false")) { block } } diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala new file mode 100644 index 000000000000..c93387bf9dcf --- /dev/null +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/wrapper/package.scala @@ -0,0 +1,24 @@ +/* + * 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 org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +package object wrapper { + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = + Dataset.ofRows(sparkSession, logicalPlan) +} diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml index 7cab49b25550..a1fcb52ce763 100755 --- a/backends-velox/pom.xml +++ b/backends-velox/pom.xml @@ -29,6 +29,74 @@ org.apache.gluten.tags.UDFTest + + iceberg + + false + + + + org.apache.gluten + gluten-iceberg + ${project.version} + + + org.apache.gluten + gluten-iceberg + ${project.version} + test-jar + test + + + org.apache.iceberg + iceberg-spark-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + provided + + + org.apache.iceberg + iceberg-spark-runtime-${sparkbundle.version}_${scala.binary.version} + ${iceberg.version} + test + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-iceberg-sources + generate-sources + + add-source + + + + ${project.basedir}/src/main-iceberg/scala + ${project.basedir}/src/main-iceberg/java + + + + + add-iceberg-test-sources + generate-test-sources + + add-test-source + + + + ${project.basedir}/src/test-iceberg/scala + ${project.basedir}/src/test-iceberg/java + + + + + + + + @@ -72,12 +140,6 @@ spark-hive_${scala.binary.version} provided
- - org.apache.gluten - gluten-substrait - ${project.version} - compile - org.apache.gluten gluten-arrow diff --git a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java index e2035455fd74..db2d08e31435 100644 --- a/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java +++ b/backends-velox/src/main/java/org/apache/gluten/columnarbatch/VeloxColumnarBatches.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.columnarbatch; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.Runtimes; @@ -59,8 +60,10 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { return input; } Preconditions.checkArgument(!isVeloxBatch(input)); - final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#toVeloxBatch"); - final long handle = ColumnarBatches.getNativeHandle(input); + final Runtime runtime = + Runtimes.contextInstance( + BackendsApiManager.getBackendName(), "VeloxColumnarBatches#toVeloxBatch"); + final long handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), input); final long outHandle = VeloxColumnarBatchJniWrapper.create(runtime).from(handle); final ColumnarBatch output = ColumnarBatches.create(outHandle); @@ -88,9 +91,13 @@ public static ColumnarBatch toVeloxBatch(ColumnarBatch input) { * Otherwise {@link UnsupportedOperationException} will be thrown. */ public static ColumnarBatch compose(ColumnarBatch... batches) { - final Runtime runtime = Runtimes.contextInstance("VeloxColumnarBatches#compose"); + final Runtime runtime = + Runtimes.contextInstance( + BackendsApiManager.getBackendName(), "VeloxColumnarBatches#compose"); final long[] handles = - Arrays.stream(batches).mapToLong(ColumnarBatches::getNativeHandle).toArray(); + Arrays.stream(batches) + .mapToLong(b -> ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), b)) + .toArray(); final long handle = VeloxColumnarBatchJniWrapper.create(runtime).compose(handles); return ColumnarBatches.create(handle); } diff --git a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java index 7250fde0fe40..d03f8816adfc 100644 --- a/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java +++ b/backends-velox/src/main/java/org/apache/gluten/metrics/IteratorMetricsJniWrapper.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.metrics; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.RuntimeAware; import org.apache.gluten.runtime.Runtimes; @@ -29,7 +30,8 @@ private IteratorMetricsJniWrapper(Runtime runtime) { } public static IteratorMetricsJniWrapper create() { - final Runtime runtime = Runtimes.contextInstance("IteratorMetrics"); + final Runtime runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "IteratorMetrics"); return new IteratorMetricsJniWrapper(runtime); } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java index 9228a2f860ae..856ddf159730 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/GlutenURLDecoder.java @@ -31,7 +31,7 @@ public class GlutenURLDecoder { *

Note: The World Wide Web Consortium * Recommendation states that UTF-8 should be used. Not doing so may introduce - * incompatibilites. + * incompatibilities. * * @param s the String to decode * @param enc The name of a supported character diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java index 2a6dcb43a052..73bfec08cf48 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBatchResizer.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.utils; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtime; import org.apache.gluten.runtime.Runtimes; import org.apache.gluten.vectorized.ColumnarBatchInIterator; @@ -28,10 +29,14 @@ public final class VeloxBatchResizer { public static ColumnarBatchOutIterator create( int minOutputBatchSize, int maxOutputBatchSize, Iterator in) { - final Runtime runtime = Runtimes.contextInstance("VeloxBatchResizer"); + final Runtime runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBatchResizer"); long outHandle = VeloxBatchResizerJniWrapper.create(runtime) - .create(minOutputBatchSize, maxOutputBatchSize, new ColumnarBatchInIterator(in)); + .create( + minOutputBatchSize, + maxOutputBatchSize, + new ColumnarBatchInIterator(BackendsApiManager.getBackendName(), in)); return new ColumnarBatchOutIterator(runtime, outHandle); } } diff --git a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java index 10179d63edb3..cdf2b195cd0c 100644 --- a/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java +++ b/backends-velox/src/main/java/org/apache/gluten/utils/VeloxBloomFilter.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.utils; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.runtime.Runtimes; import org.apache.commons.io.IOUtils; @@ -30,7 +31,8 @@ public class VeloxBloomFilter extends BloomFilter { private final VeloxBloomFilterJniWrapper jni = - VeloxBloomFilterJniWrapper.create(Runtimes.contextInstance("VeloxBloomFilter")); + VeloxBloomFilterJniWrapper.create( + Runtimes.contextInstance(BackendsApiManager.getBackendName(), "VeloxBloomFilter")); private final long handle; private VeloxBloomFilter(byte[] data) { diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala index 251b93cc7cec..6bf4b6a4e256 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxBackend.scala @@ -18,7 +18,7 @@ package org.apache.gluten.backendsapi.velox import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component.BuildInfo import org.apache.gluten.backendsapi._ import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.exception.GlutenNotSupportException @@ -35,12 +35,13 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Alias, CumeDist, DenseRank, Descending, Expression, Lag, Lead, NamedExpression, NthValue, NTile, PercentRank, RangeFrame, Rank, RowNumber, SortOrder, SpecialFrameBoundary, SpecifiedWindowFrame} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, ApproximatePercentile, Percentile} import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CharVarcharUtils} import org.apache.spark.sql.execution.{ColumnarCachedBatchSerializer, SparkPlan} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} import org.apache.spark.sql.hive.execution.HiveFileFormat import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -51,11 +52,11 @@ import scala.util.control.Breaks.breakable class VeloxBackend extends SubstraitBackend { import VeloxBackend._ + override def name(): String = VeloxBackend.BACKEND_NAME - override def defaultBatchType: Convention.BatchType = VeloxBatch + override def buildInfo(): BuildInfo = + BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) override def convFuncOverride(): ConventionFunc.Override = new ConvFunc() - override def buildInfo(): Backend.BuildInfo = - Backend.BuildInfo("Velox", VELOX_BRANCH, VELOX_REVISION, VELOX_REVISION_TIME) override def iteratorApi(): IteratorApi = new VeloxIteratorApi override def sparkPlanExecApi(): SparkPlanExecApi = new VeloxSparkPlanExecApi override def transformerApi(): TransformerApi = new VeloxTransformerApi @@ -72,6 +73,8 @@ object VeloxBackend { private class ConvFunc() extends ConventionFunc.Override { override def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + case a: AdaptiveSparkPlanExec if a.supportsColumnar => + VeloxBatch case i: InMemoryTableScanExec if i.supportsColumnar && i.relation.cacheBuilder.serializer .isInstanceOf[ColumnarCachedBatchSerializer] => @@ -81,85 +84,103 @@ object VeloxBackend { } object VeloxBackendSettings extends BackendSettingsApi { - val SHUFFLE_SUPPORTED_CODEC = Set("lz4", "zstd") val GLUTEN_VELOX_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".udfLibraryPaths" val GLUTEN_VELOX_DRIVER_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".driver.udfLibraryPaths" val GLUTEN_VELOX_INTERNAL_UDF_LIB_PATHS = VeloxBackend.CONF_PREFIX + ".internal.udfLibraryPaths" val GLUTEN_VELOX_UDF_ALLOW_TYPE_CONVERSION = VeloxBackend.CONF_PREFIX + ".udfAllowTypeConversion" - val MAXIMUM_BATCH_SIZE: Int = 32768 + /** The columnar-batch type this backend is by default using. */ + override def primaryBatchType: Convention.BatchType = VeloxBatch - override def validateScan( + override def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = { - val filteredRootPaths = distinctRootPaths(rootPaths) - if ( - filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper - .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray) - ) { - return ValidationResult.failed( - s"Scheme of [$filteredRootPaths] is not supported by registered file systems.") - } - // Validate if all types are supported. - def validateTypes(validatorFunc: PartialFunction[StructField, String]): ValidationResult = { - // Collect unsupported types. - val unsupportedDataTypeReason = fields.collect(validatorFunc) - if (unsupportedDataTypeReason.isEmpty) { - ValidationResult.succeeded + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = { + + def validateScheme(): Option[String] = { + val filteredRootPaths = distinctRootPaths(rootPaths) + if ( + filteredRootPaths.nonEmpty && !VeloxFileSystemValidationJniWrapper + .allSupportedByRegisteredFileSystems(filteredRootPaths.toArray) + ) { + Some(s"Scheme of [$filteredRootPaths] is not supported by registered file systems.") } else { - ValidationResult.failed( - s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.") + None } } - format match { - case ParquetReadFormat => - val typeValidator: PartialFunction[StructField, String] = { - // Parquet timestamp is not fully supported yet - case StructField(_, TimestampType, _, _) - if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled => - "TimestampType" - } - validateTypes(typeValidator) - case DwrfReadFormat => ValidationResult.succeeded - case OrcReadFormat => - if (!GlutenConfig.getConf.veloxOrcScanEnabled) { - ValidationResult.failed(s"Velox ORC scan is turned off.") + def validateFormat(): Option[String] = { + def validateTypes(validatorFunc: PartialFunction[StructField, String]): Option[String] = { + // Collect unsupported types. + val unsupportedDataTypeReason = fields.collect(validatorFunc) + if (unsupportedDataTypeReason.nonEmpty) { + Some( + s"Found unsupported data type in $format: ${unsupportedDataTypeReason.mkString(", ")}.") } else { + None + } + } + + def isCharType(stringType: StringType, metadata: Metadata): Boolean = { + val charTypePattern = "char\\((\\d+)\\)".r + GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern + .findFirstIn( + CharVarcharUtils + .getRawTypeString(metadata) + .getOrElse(stringType.catalogString)) + .isDefined + } + + format match { + case ParquetReadFormat => val typeValidator: PartialFunction[StructField, String] = { - case StructField(_, arrayType: ArrayType, _, _) - if arrayType.elementType.isInstanceOf[StructType] => - "StructType as element in ArrayType" - case StructField(_, arrayType: ArrayType, _, _) - if arrayType.elementType.isInstanceOf[ArrayType] => - "ArrayType as element in ArrayType" - case StructField(_, mapType: MapType, _, _) - if mapType.keyType.isInstanceOf[StructType] => - "StructType as Key in MapType" - case StructField(_, mapType: MapType, _, _) - if mapType.valueType.isInstanceOf[ArrayType] => - "ArrayType as Value in MapType" - case StructField(_, stringType: StringType, _, metadata) - if isCharType(stringType, metadata) => - CharVarcharUtils.getRawTypeString(metadata) + " not support" - case StructField(_, TimestampType, _, _) => "TimestampType not support" + // Parquet timestamp is not fully supported yet + case StructField(_, TimestampType, _, _) + if GlutenConfig.getConf.forceParquetTimestampTypeScanFallbackEnabled => + "TimestampType(force fallback)" } - validateTypes(typeValidator) - } - case _ => ValidationResult.failed(s"Unsupported file format for $format.") + val parquetOptions = new ParquetOptions(CaseInsensitiveMap(properties), SQLConf.get) + if (parquetOptions.mergeSchema) { + // https://github.com/apache/incubator-gluten/issues/7174 + Some(s"not support when merge schema is true") + } else { + validateTypes(typeValidator) + } + case DwrfReadFormat => None + case OrcReadFormat => + if (!GlutenConfig.getConf.veloxOrcScanEnabled) { + Some(s"Velox ORC scan is turned off, ${GlutenConfig.VELOX_ORC_SCAN_ENABLED.key}") + } else { + val typeValidator: PartialFunction[StructField, String] = { + case StructField(_, arrayType: ArrayType, _, _) + if arrayType.elementType.isInstanceOf[StructType] => + "StructType as element in ArrayType" + case StructField(_, arrayType: ArrayType, _, _) + if arrayType.elementType.isInstanceOf[ArrayType] => + "ArrayType as element in ArrayType" + case StructField(_, mapType: MapType, _, _) + if mapType.keyType.isInstanceOf[StructType] => + "StructType as Key in MapType" + case StructField(_, mapType: MapType, _, _) + if mapType.valueType.isInstanceOf[ArrayType] => + "ArrayType as Value in MapType" + case StructField(_, stringType: StringType, _, metadata) + if isCharType(stringType, metadata) => + CharVarcharUtils.getRawTypeString(metadata) + "(force fallback)" + case StructField(_, TimestampType, _, _) => "TimestampType" + } + validateTypes(typeValidator) + } + case _ => Some(s"Unsupported file format for $format.") + } } - } - def isCharType(stringType: StringType, metadata: Metadata): Boolean = { - val charTypePattern = "char\\((\\d+)\\)".r - GlutenConfig.getConf.forceOrcCharTypeScanFallbackEnabled && charTypePattern - .findFirstIn( - CharVarcharUtils - .getRawTypeString(metadata) - .getOrElse(stringType.catalogString)) - .isDefined + validateScheme().orElse(validateFormat()) match { + case Some(reason) => ValidationResult.failed(reason) + case _ => ValidationResult.succeeded + } } def distinctRootPaths(paths: Seq[String]): Seq[String] = { @@ -216,15 +237,26 @@ object VeloxBackendSettings extends BackendSettingsApi { // Validate if all types are supported. def validateDataTypes(): Option[String] = { - val unsupportedTypes = fields.flatMap { - field => - field.dataType match { - case _: StructType => Some("StructType") - case _: ArrayType => Some("ArrayType") - case _: MapType => Some("MapType") - case _: YearMonthIntervalType => Some("YearMonthIntervalType") + val unsupportedTypes = format match { + case _: ParquetFileFormat => + fields.flatMap { + case StructField(_, _: YearMonthIntervalType, _, _) => + Some("YearMonthIntervalType") + case StructField(_, _: StructType, _, _) => + Some("StructType") case _ => None } + case _ => + fields.flatMap { + field => + field.dataType match { + case _: StructType => Some("StructType") + case _: ArrayType => Some("ArrayType") + case _: MapType => Some("MapType") + case _: YearMonthIntervalType => Some("YearMonthIntervalType") + case _ => None + } + } } if (unsupportedTypes.nonEmpty) { Some(unsupportedTypes.mkString("Found unsupported type:", ",", "")) @@ -368,8 +400,10 @@ object VeloxBackendSettings extends BackendSettingsApi { case _ => } windowExpression.windowFunction match { - case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | - _: NthValue | _: NTile | _: Lag | _: Lead => + case _: RowNumber | _: Rank | _: CumeDist | _: DenseRank | _: PercentRank | _: NTile => + case nv: NthValue if !nv.input.foldable => + case l: Lag if !l.input.foldable => + case l: Lead if !l.input.foldable => case aggrExpr: AggregateExpression if !aggrExpr.aggregateFunction.isInstanceOf[ApproximatePercentile] && !aggrExpr.aggregateFunction.isInstanceOf[Percentile] => diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala index 320d1f366c23..26bc108c15d9 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala @@ -39,9 +39,7 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.{ExecutorManager, SerializableConfiguration, SparkDirectoryUtil} - -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.util.{ExecutorManager, SparkDirectoryUtil} import java.lang.{Long => JLong} import java.nio.charset.StandardCharsets @@ -57,8 +55,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo = { + properties: Map[String, String]): SplitInfo = { partition match { case f: FilePartition => val ( @@ -69,7 +66,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { modificationTimes, partitionColumns, metadataColumns) = - constructSplitInfo(partitionSchema, f.files, metadataColumnNames, serializableHadoopConf) + constructSplitInfo(partitionSchema, f.files, metadataColumnNames) val preferredLocations = SoftAffinity.getFilePartitionLocations(f) LocalFilesBuilder.makeLocalFiles( @@ -112,8 +109,7 @@ class VeloxIteratorApi extends IteratorApi with Logging { private def constructSplitInfo( schema: StructType, files: Array[PartitionedFile], - metadataColumnNames: Seq[String], - serializableHadoopConf: SerializableConfiguration) = { + metadataColumnNames: Seq[String]) = { val paths = new JArrayList[String]() val starts = new JArrayList[JLong] val lengths = new JArrayList[JLong]() @@ -125,15 +121,9 @@ class VeloxIteratorApi extends IteratorApi with Logging { file => // The "file.filePath" in PartitionedFile is not the original encoded path, so the decoded // path is incorrect in some cases and here fix the case of ' ' by using GlutenURLDecoder - var filePath = file.filePath.toString - if (filePath.startsWith("viewfs")) { - val viewPath = new Path(filePath) - val viewFileSystem = FileSystem.get(viewPath.toUri, serializableHadoopConf.value) - filePath = viewFileSystem.resolvePath(viewPath).toString - } paths.add( GlutenURLDecoder - .decode(filePath, StandardCharsets.UTF_8.name())) + .decode(file.filePath.toString, StandardCharsets.UTF_8.name())) starts.add(JLong.valueOf(file.start)) lengths.add(JLong.valueOf(file.length)) val (fileSize, modificationTime) = @@ -193,9 +183,9 @@ class VeloxIteratorApi extends IteratorApi with Logging { val columnarNativeIterators = new JArrayList[ColumnarBatchInIterator](inputIterators.map { - iter => new ColumnarBatchInIterator(iter.asJava) + iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava) }.asJava) - val transKernel = NativePlanEvaluator.create() + val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName) val splitInfoByteArray = inputPartition .asInstanceOf[GlutenPartition] @@ -245,10 +235,10 @@ class VeloxIteratorApi extends IteratorApi with Logging { ExecutorManager.tryTaskSet(numaBindingInfo) - val transKernel = NativePlanEvaluator.create() + val transKernel = NativePlanEvaluator.create(BackendsApiManager.getBackendName) val columnarNativeIterator = new JArrayList[ColumnarBatchInIterator](inputIterators.map { - iter => new ColumnarBatchInIterator(iter.asJava) + iter => new ColumnarBatchInIterator(BackendsApiManager.getBackendName, iter.asJava) }.asJava) val spillDirPath = SparkDirectoryUtil .get() diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala index 850509db3e91..175e34177a5d 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxListenerApi.scala @@ -31,8 +31,9 @@ import org.apache.spark.{HdfsConfGenerator, SparkConf, SparkContext} import org.apache.spark.api.plugin.PluginContext import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit +import org.apache.spark.sql.execution.ColumnarCachedBatchSerializer import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules -import org.apache.spark.sql.execution.datasources.velox.{VeloxOrcWriterInjects, VeloxParquetWriterInjects, VeloxRowSplitter} +import org.apache.spark.sql.execution.datasources.velox.{VeloxParquetWriterInjects, VeloxRowSplitter} import org.apache.spark.sql.expression.UDFResolver import org.apache.spark.sql.internal.{GlutenConfigUtil, StaticSQLConf} import org.apache.spark.util.{SparkDirectoryUtil, SparkResourceUtil} @@ -75,7 +76,7 @@ class VeloxListenerApi extends ListenerApi with Logging { if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, defaultValue = false)) { conf.set( StaticSQLConf.SPARK_CACHE_SERIALIZER.key, - "org.apache.spark.sql.execution.ColumnarCachedBatchSerializer") + classOf[ColumnarCachedBatchSerializer].getName) } // Static initializers for driver. @@ -89,7 +90,7 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = true) - initialize(conf) + initialize(conf, isDriver = true) UdfJniWrapper.registerFunctionSignatures() } @@ -116,16 +117,16 @@ class VeloxListenerApi extends ListenerApi with Logging { SparkDirectoryUtil.init(conf) UDFResolver.resolveUdfConf(conf, isDriver = false) - initialize(conf) + initialize(conf, isDriver = false) } override def onExecutorShutdown(): Unit = shutdown() - private def initialize(conf: SparkConf): Unit = { + private def initialize(conf: SparkConf, isDriver: Boolean): Unit = { // Force batch type initializations. - VeloxBatch.getClass - ArrowJavaBatch.getClass - ArrowNativeBatch.getClass + VeloxBatch.ensureRegistered() + ArrowJavaBatch.ensureRegistered() + ArrowNativeBatch.ensureRegistered() // Sets this configuration only once, since not undoable. if (conf.getBoolean(GlutenConfig.GLUTEN_DEBUG_KEEP_JNI_WORKSPACE, defaultValue = false)) { @@ -157,11 +158,16 @@ class VeloxListenerApi extends ListenerApi with Logging { } // Initial native backend with configurations. - val parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) - NativeBackendInitializer.initializeBackend(parsed) + var parsed = GlutenConfigUtil.parseConfig(conf.getAll.toMap) + + // Workaround for https://github.com/apache/incubator-gluten/issues/7837 + if (isDriver && !inLocalMode(conf)) { + parsed += (GlutenConfig.COLUMNAR_VELOX_CACHE_ENABLED.key -> "false") + } + NativeBackendInitializer.forBackend(VeloxBackend.BACKEND_NAME).initialize(parsed) // Inject backend-specific implementations to override spark classes. - GlutenFormatFactory.register(new VeloxParquetWriterInjects, new VeloxOrcWriterInjects) + GlutenFormatFactory.register(new VeloxParquetWriterInjects) GlutenFormatFactory.injectPostRuleFactory( session => GlutenWriterColumnarRules.NativeWritePostRule(session)) GlutenFormatFactory.register(new VeloxRowSplitter()) diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala index 10b0c493c10a..934b680382ea 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxMetricsApi.scala @@ -22,7 +22,8 @@ import org.apache.gluten.substrait.{AggregationParams, JoinParams} import org.apache.spark.SparkContext import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ColumnarInputAdapter, SparkPlan} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import java.lang.{Long => JLong} @@ -38,18 +39,37 @@ class VeloxMetricsApi extends MetricsApi with Logging { } override def genInputIteratorTransformerMetrics( - sparkContext: SparkContext): Map[String, SQLMetric] = { + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] = { + def metricsPlan(plan: SparkPlan): SparkPlan = { + plan match { + case ColumnarInputAdapter(child) => metricsPlan(child) + case q: QueryStageExec => metricsPlan(q.plan) + case _ => plan + } + } + + val outputMetrics = if (forBroadcast) { + metricsPlan(child).metrics + .filterKeys(key => key.equals("numOutputRows") || key.equals("outputVectors")) + } else { + Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors") + ) + } + Map( "cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count"), - "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator"), - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "outputVectors" -> SQLMetrics.createMetric(sparkContext, "number of output vectors") - ) + "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of input iterator") + ) ++ outputMetrics } override def genInputIteratorTransformerMetricsUpdater( - metrics: Map[String, SQLMetric]): MetricsUpdater = { - InputIteratorMetricsUpdater(metrics) + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater = { + InputIteratorMetricsUpdater(metrics, forBroadcast) } override def genBatchScanTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = @@ -562,4 +582,15 @@ class VeloxMetricsApi extends MetricsApi with Logging { override def genSampleTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = new SampleMetricsUpdater(metrics) + + override def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map( + "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), + "inputVectors" -> SQLMetrics.createMetric(sparkContext, "number of input vectors"), + "inputBytes" -> SQLMetrics.createSizeMetric(sparkContext, "number of input bytes"), + "wallNanos" -> SQLMetrics.createNanoTimingMetric(sparkContext, "time of union"), + "cpuCount" -> SQLMetrics.createMetric(sparkContext, "cpu wall time count") + ) + + override def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater = + new UnionMetricsUpdater(metrics) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala index a838c463c390..7337be573710 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxRuleApi.scala @@ -16,60 +16,93 @@ */ package org.apache.gluten.backendsapi.velox +import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.RuleApi -import org.apache.gluten.datasource.ArrowConvertorRule +import org.apache.gluten.columnarbatch.VeloxBatch import org.apache.gluten.extension._ import org.apache.gluten.extension.columnar._ -import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast, TransformPreOverrides} -import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.MiscColumnarRules.{RemoveGlutenTableCacheColumnarToRow, RemoveTopmostColumnarToRow, RewriteSubqueryBroadcast} +import org.apache.gluten.extension.columnar.enumerated.{RasOffload, RemoveSort} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, RoughCoster, RoughCoster2} +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicTransform} +import org.apache.gluten.extension.columnar.offload.{OffloadExchange, OffloadJoin, OffloadOthers} +import org.apache.gluten.extension.columnar.rewrite._ import org.apache.gluten.extension.columnar.transition.{InsertTransitions, RemoveTransitions} -import org.apache.gluten.extension.injector.{RuleInjector, SparkInjector} +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.columnar.validator.Validators.ValidatorBuilderImplicits +import org.apache.gluten.extension.injector.{Injector, SparkInjector} import org.apache.gluten.extension.injector.GlutenInjector.{LegacyInjector, RasInjector} import org.apache.gluten.sql.shims.SparkShimLoader -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, GlutenFallbackReporter} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.execution.datasources.WriteFilesExec +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.joins.BaseJoinExec +import org.apache.spark.sql.execution.python.EvalPythonExec +import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.hive.HiveTableScanExecTransformer class VeloxRuleApi extends RuleApi { import VeloxRuleApi._ - override def injectRules(injector: RuleInjector): Unit = { + override def injectRules(injector: Injector): Unit = { injectSpark(injector.spark) injectLegacy(injector.gluten.legacy) injectRas(injector.gluten.ras) } } -private object VeloxRuleApi { - def injectSpark(injector: SparkInjector): Unit = { +object VeloxRuleApi { + private def injectSpark(injector: SparkInjector): Unit = { // Inject the regular Spark rules directly. injector.injectOptimizerRule(CollectRewriteRule.apply) injector.injectOptimizerRule(HLLRewriteRule.apply) injector.injectPostHocResolutionRule(ArrowConvertorRule.apply) } - def injectLegacy(injector: LegacyInjector): Unit = { - // Gluten columnar: Transform rules. - injector.injectTransform(_ => RemoveTransitions) - injector.injectTransform(_ => PushDownInputFileExpression.PreOffload) - injector.injectTransform(c => FallbackOnANSIMode.apply(c.session)) - injector.injectTransform(c => FallbackMultiCodegens.apply(c.session)) - injector.injectTransform(_ => RewriteSubqueryBroadcast()) - injector.injectTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) - injector.injectTransform(c => ArrowScanReplaceRule.apply(c.session)) - injector.injectTransform(_ => RewriteSparkPlanRulesManager()) - injector.injectTransform(_ => AddFallbackTagRule()) - injector.injectTransform(_ => TransformPreOverrides()) - injector.injectTransform(c => PartialProjectRule.apply(c.session)) - injector.injectTransform(_ => RemoveNativeWriteFilesSortAndProject()) - injector.injectTransform(c => RewriteTransformer.apply(c.session)) - injector.injectTransform(_ => PushDownFilterToScan) - injector.injectTransform(_ => PushDownInputFileExpression.PostOffload) - injector.injectTransform(_ => EnsureLocalSortRequirements) - injector.injectTransform(_ => EliminateLocalSort) - injector.injectTransform(_ => CollapseProjectExecTransformer) - injector.injectTransform(c => FlushableHashAggregateRule.apply(c.session)) - injector.injectTransform(c => InsertTransitions(c.outputsColumnar)) + private def injectLegacy(injector: LegacyInjector): Unit = { + // Legacy: Pre-transform rules. + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => FallbackMultiCodegens.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) + injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) + + // Legacy: The legacy transform rule. + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .fallbackByNativeValidation() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + val offloads = Seq(OffloadOthers(), OffloadExchange(), OffloadJoin()) + injector.injectTransform( + c => HeuristicTransform.Single(validatorBuilder(c.glutenConf), rewrites, offloads)) + + // Legacy: Post-transform rules. + injector.injectPostTransform(_ => UnionTransformerRule()) + injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) // Gluten columnar: Fallback policies. injector.injectFallbackPolicy( @@ -80,45 +113,91 @@ private object VeloxRuleApi { SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() .foreach(each => injector.injectPost(c => each(c.session))) - injector.injectPost(c => ColumnarCollapseTransformStages(c.conf)) + injector.injectPost(c => ColumnarCollapseTransformStages(c.glutenConf)) // Gluten columnar: Final rules. injector.injectFinal(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.injectFinal(c => GlutenFallbackReporter(c.conf, c.session)) + injector.injectFinal(c => GlutenFallbackReporter(c.glutenConf, c.session)) injector.injectFinal(_ => RemoveFallbackTagRule()) } - def injectRas(injector: RasInjector): Unit = { + private def injectRas(injector: RasInjector): Unit = { // Gluten RAS: Pre rules. - injector.inject(_ => RemoveTransitions) - injector.inject(_ => PushDownInputFileExpression.PreOffload) - injector.inject(c => FallbackOnANSIMode.apply(c.session)) - injector.inject(_ => RewriteSubqueryBroadcast()) - injector.inject(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) - injector.inject(c => ArrowScanReplaceRule.apply(c.session)) + injector.injectPreTransform(_ => RemoveTransitions) + injector.injectPreTransform(_ => PushDownInputFileExpression.PreOffload) + injector.injectPreTransform(c => FallbackOnANSIMode.apply(c.session)) + injector.injectPreTransform(c => MergeTwoPhasesHashBaseAggregate(c.session)) + injector.injectPreTransform(_ => RewriteSubqueryBroadcast()) + injector.injectPreTransform(c => BloomFilterMightContainJointRewriteRule.apply(c.session)) + injector.injectPreTransform(c => ArrowScanReplaceRule.apply(c.session)) // Gluten RAS: The RAS rule. - injector.inject(c => EnumeratedTransform(c.session, c.outputsColumnar)) + val validatorBuilder: GlutenConfig => Validator = conf => + Validator + .builder() + .fallbackByHint() + .fallbackIfScanOnlyWithFilterPushed(conf.enableScanOnly) + .fallbackComplexExpressions() + .fallbackByBackendSettings() + .fallbackByUserOptions() + .fallbackByTestInjects() + .build() + val rewrites = + Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) + injector.injectCoster(_ => LegacyCoster) + injector.injectCoster(_ => RoughCoster) + injector.injectCoster(_ => RoughCoster2) + injector.injectRasRule(_ => RemoveSort) + val offloads: Seq[RasOffload] = Seq( + RasOffload.from[Exchange](OffloadExchange()), + RasOffload.from[BaseJoinExec](OffloadJoin()), + RasOffload.from[FilterExec](OffloadOthers()), + RasOffload.from[ProjectExec](OffloadOthers()), + RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()), + RasOffload.from[DataSourceScanExec](OffloadOthers()), + RasOffload.from(HiveTableScanExecTransformer.isHiveTableScan(_))(OffloadOthers()), + RasOffload.from[CoalesceExec](OffloadOthers()), + RasOffload.from[HashAggregateExec](OffloadOthers()), + RasOffload.from[SortAggregateExec](OffloadOthers()), + RasOffload.from[ObjectHashAggregateExec](OffloadOthers()), + RasOffload.from[UnionExec](OffloadOthers()), + RasOffload.from[ExpandExec](OffloadOthers()), + RasOffload.from[WriteFilesExec](OffloadOthers()), + RasOffload.from[SortExec](OffloadOthers()), + RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()), + RasOffload.from[WindowExec](OffloadOthers()), + RasOffload.from(SparkShimLoader.getSparkShims.isWindowGroupLimitExec(_))(OffloadOthers()), + RasOffload.from[LimitExec](OffloadOthers()), + RasOffload.from[GenerateExec](OffloadOthers()), + RasOffload.from[EvalPythonExec](OffloadOthers()), + RasOffload.from[SampleExec](OffloadOthers()) + ) + offloads.foreach( + offload => + injector.injectRasRule( + c => RasOffload.Rule(offload, validatorBuilder(c.glutenConf), rewrites))) // Gluten RAS: Post rules. - injector.inject(_ => RemoveTransitions) - injector.inject(c => PartialProjectRule.apply(c.session)) - injector.inject(_ => RemoveNativeWriteFilesSortAndProject()) - injector.inject(c => RewriteTransformer.apply(c.session)) - injector.inject(_ => PushDownFilterToScan) - injector.inject(_ => PushDownInputFileExpression.PostOffload) - injector.inject(_ => EnsureLocalSortRequirements) - injector.inject(_ => EliminateLocalSort) - injector.inject(_ => CollapseProjectExecTransformer) - injector.inject(c => FlushableHashAggregateRule.apply(c.session)) - injector.inject(c => InsertTransitions(c.outputsColumnar)) - injector.inject(c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) + injector.injectPostTransform(_ => RemoveTransitions) + injector.injectPostTransform(_ => UnionTransformerRule()) + injector.injectPostTransform(c => PartialProjectRule.apply(c.session)) + injector.injectPostTransform(_ => RemoveNativeWriteFilesSortAndProject()) + injector.injectPostTransform(c => RewriteTransformer.apply(c.session)) + injector.injectPostTransform(_ => PushDownFilterToScan) + injector.injectPostTransform(_ => PushDownInputFileExpression.PostOffload) + injector.injectPostTransform(_ => EnsureLocalSortRequirements) + injector.injectPostTransform(_ => EliminateLocalSort) + injector.injectPostTransform(_ => CollapseProjectExecTransformer) + injector.injectPostTransform(c => FlushableHashAggregateRule.apply(c.session)) + injector.injectPostTransform(c => InsertTransitions.create(c.outputsColumnar, VeloxBatch)) + injector.injectPostTransform( + c => RemoveTopmostColumnarToRow(c.session, c.ac.isAdaptiveContext())) SparkShimLoader.getSparkShims .getExtendedColumnarPostRules() - .foreach(each => injector.inject(c => each(c.session))) - injector.inject(c => ColumnarCollapseTransformStages(c.conf)) - injector.inject(c => RemoveGlutenTableCacheColumnarToRow(c.session)) - injector.inject(c => GlutenFallbackReporter(c.conf, c.session)) - injector.inject(_ => RemoveFallbackTagRule()) + .foreach(each => injector.injectPostTransform(c => each(c.session))) + injector.injectPostTransform(c => ColumnarCollapseTransformStages(c.glutenConf)) + injector.injectPostTransform(c => RemoveGlutenTableCacheColumnarToRow(c.session)) + injector.injectPostTransform(c => GlutenFallbackReporter(c.glutenConf, c.session)) + injector.injectPostTransform(_ => RemoveFallbackTagRule()) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala index 81564a440117..d837ac423407 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxSparkPlanExecApi.scala @@ -153,7 +153,8 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { left.dataType.isInstanceOf[DecimalType] && right.dataType .isInstanceOf[DecimalType] && !SQLConf.get.decimalOperationsAllowPrecisionLoss ) { - val newName = "not_allow_precision_loss_" + // https://github.com/facebookincubator/velox/pull/10383 + val newName = substraitExprName + "_deny_precision_loss" GenericExpressionTransformer(newName, Seq(left, right), original) } else { GenericExpressionTransformer(substraitExprName, Seq(left, right), original) @@ -631,7 +632,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { } numOutputRows += serialized.map(_.getNumRows).sum dataSize += rawSize - ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized)) + ColumnarBuildSideRelation(child.output, serialized.map(_.getSerialized), mode) } override def doCanonicalizeForBroadcastMode(mode: BroadcastMode): BroadcastMode = { @@ -723,30 +724,29 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { val trimParaSepStr = "\u2029" // Needs to be trimmed for casting to float/double/decimal val trimSpaceStr = ('\u0000' to '\u0020').toList.mkString + // ISOControl characters, refer java.lang.Character.isISOControl(int) + val isoControlStr = (('\u0000' to '\u001F') ++ ('\u007F' to '\u009F')).toList.mkString // scalastyle:on nonascii - c.dataType match { - case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] => - c - case FloatType | DoubleType | _: DecimalType => - c.child.dataType match { - case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode => - val trimNode = StringTrim(c.child, Some(Literal(trimSpaceStr))) - c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast] - case _ => - c - } - case _ => - c.child.dataType match { - case StringType if GlutenConfig.getConf.castFromVarcharAddTrimNode => - val trimNode = StringTrim( - c.child, - Some( - Literal(trimWhitespaceStr + - trimSpaceSepStr + trimLineSepStr + trimParaSepStr))) - c.withNewChildren(Seq(trimNode)).asInstanceOf[Cast] - case _ => - c + if (GlutenConfig.getConf.castFromVarcharAddTrimNode && c.child.dataType == StringType) { + val trimStr = c.dataType match { + case BinaryType | _: ArrayType | _: MapType | _: StructType | _: UserDefinedType[_] => + None + case FloatType | DoubleType | _: DecimalType => + Some(trimSpaceStr) + case _ => + Some( + (trimWhitespaceStr + trimSpaceSepStr + trimLineSepStr + + trimParaSepStr + isoControlStr).toSet.mkString + ) + } + trimStr + .map { + trim => + c.withNewChildren(Seq(StringTrim(c.child, Some(Literal(trim))))).asInstanceOf[Cast] } + .getOrElse(c) + } else { + c } } @@ -807,7 +807,7 @@ class VeloxSparkPlanExecApi extends SparkPlanExecApi { override def maybeCollapseTakeOrderedAndProject(plan: SparkPlan): SparkPlan = { // This to-top-n optimization assumes exchange operators were already placed in input plan. plan.transformUp { - case p @ LimitTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) => + case p @ LimitExecTransformer(SortExecTransformer(sortOrder, _, child, _), 0, count) => val global = child.outputPartitioning.satisfies(AllTuples) val topN = TopNTransformer(count, sortOrder, global, child) if (topN.doValidate().ok()) { diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala index 1687f24ce35e..c6d2bc065879 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxTransformerApi.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.backendsapi.velox -import org.apache.gluten.backendsapi.TransformerApi +import org.apache.gluten.backendsapi.{BackendsApiManager, TransformerApi} import org.apache.gluten.execution.WriteFilesExecTransformer import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.runtime.Runtimes @@ -87,7 +87,9 @@ class VeloxTransformerApi extends TransformerApi with Logging { override def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String = { TaskResources.runUnsafe { val jniWrapper = PlanEvaluatorJniWrapper.create( - Runtimes.contextInstance("VeloxTransformerApi#getNativePlanString")) + Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "VeloxTransformerApi#getNativePlanString")) jniWrapper.nativePlanString(substraitPlan, details) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala index ddf77e5fa3d4..9b3001366198 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxValidatorApi.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.backendsapi.velox -import org.apache.gluten.backendsapi.ValidatorApi +import org.apache.gluten.backendsapi.{BackendsApiManager, ValidatorApi} import org.apache.gluten.extension.ValidationResult import org.apache.gluten.substrait.plan.PlanNode import org.apache.gluten.validate.NativePlanValidationInfo @@ -38,7 +38,7 @@ class VeloxValidatorApi extends ValidatorApi { override def doNativeValidateWithFailureReason(plan: PlanNode): ValidationResult = { TaskResources.runUnsafe { - val validator = NativePlanEvaluator.create() + val validator = NativePlanEvaluator.create(BackendsApiManager.getBackendName) asValidationResult(validator.doNativeValidateWithFailureReason(plan.toProtobuf.toByteArray)) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala index 0c7600c856b1..5d9a78d31874 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/columnarbatch/VeloxBatch.scala @@ -20,8 +20,10 @@ import org.apache.gluten.execution.{ArrowColumnarToVeloxColumnarExec, RowToVelox import org.apache.gluten.extension.columnar.transition.{Convention, Transition} object VeloxBatch extends Convention.BatchType { - fromRow(RowToVeloxColumnarExec.apply) - toRow(VeloxColumnarToRowExec.apply) - fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply) - toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty) + override protected def registerTransitions(): Unit = { + fromRow(RowToVeloxColumnarExec.apply) + toRow(VeloxColumnarToRowExec.apply) + fromBatch(ArrowBatches.ArrowNativeBatch, ArrowColumnarToVeloxColumnarExec.apply) + toBatch(ArrowBatches.ArrowNativeBatch, Transition.empty) + } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala index fe0d0eb0f8f2..7ea67feb1f8d 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/datasource/VeloxDataSourceUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.datasource +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes import org.apache.gluten.utils.ArrowAbiUtil @@ -38,7 +39,7 @@ object VeloxDataSourceUtil { def readSchema(file: FileStatus): Option[StructType] = { val allocator = ArrowBufferAllocators.contextInstance() - val runtime = Runtimes.contextInstance("VeloxWriter") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val dsHandle = datasourceJniWrapper.init(file.getPath.toString, -1, new util.HashMap[String, String]()) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala index d993e399dbf4..576f3a2cb205 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/ColumnarPartialProjectExec.scala @@ -17,9 +17,11 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.expression.{ArrowProjection, ExpressionUtils} -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.sql.shims.SparkShimLoader @@ -31,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeRef import org.apache.spark.sql.execution.{ExplainUtils, ProjectExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.hive.HiveUdfUtil -import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, DataType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, NullType, ShortType, StringType, TimestampType, YearMonthIntervalType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} import scala.collection.mutable.ListBuffer @@ -51,7 +53,7 @@ import scala.collection.mutable.ListBuffer case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( replacedAliasUdf: Seq[Alias]) extends UnaryExecNode - with GlutenPlan { + with ValidatablePlan { private val projectAttributes: ListBuffer[Attribute] = ListBuffer() private val projectIndexInChild: ListBuffer[Int] = ListBuffer() @@ -73,6 +75,10 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( override def output: Seq[Attribute] = child.output ++ replacedAliasUdf.map(_.toAttribute) + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + final override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( s"${this.getClass.getSimpleName} doesn't support doExecute") @@ -82,8 +88,6 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( replacedAliasUdf :: Nil } - final override val supportsColumnar: Boolean = true - private def validateExpression(expr: Expression): Boolean = { expr.deterministic && !expr.isInstanceOf[LambdaFunction] && expr.children .forall(validateExpression) @@ -184,7 +188,8 @@ case class ColumnarPartialProjectExec(original: ProjectExec, child: SparkPlan)( Iterator.empty } else { val start = System.currentTimeMillis() - val childData = ColumnarBatches.select(batch, projectIndexInChild.toArray) + val childData = ColumnarBatches + .select(BackendsApiManager.getBackendName, batch, projectIndexInChild.toArray) val projectedBatch = getProjectedBatchArrow(childData, c2a, a2c) val batchIterator = projectedBatch.map { diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala index fe5e0d92d6d5..9a6ba3a220f1 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/HashAggregateExecTransformer.scala @@ -72,7 +72,7 @@ abstract class HashAggregateExecTransformer( val aggParams = new AggregationParams val operatorId = context.nextOperatorId(this.nodeName) val relNode = getAggRel(context, operatorId, aggParams, childCtx.root) - TransformContext(childCtx.outputAttributes, output, relNode) + TransformContext(output, relNode) } // Return whether the outputs partial aggregation should be combined for Velox computing. @@ -214,7 +214,7 @@ abstract class HashAggregateExecTransformer( VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction) ) aggregateNodeList.add(aggFunctionNode) - case Final => + case Final | Complete => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode), childrenNodeList, @@ -242,7 +242,7 @@ abstract class HashAggregateExecTransformer( aggregateFunction.inputAggBufferAttributes.head.nullable) ) aggregateNodeList.add(partialNode) - case Final => + case Final | Complete => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( VeloxAggregateFunctionsBuilder.create(args, aggregateFunction, aggregateMode), childrenNodeList, @@ -275,7 +275,7 @@ abstract class HashAggregateExecTransformer( expression.mode match { case Partial | PartialMerge => typeNodeList.add(VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction)) - case Final => + case Final | Complete => typeNodeList.add( ConverterUtils .getTypeNode(aggregateFunction.dataType, aggregateFunction.nullable)) @@ -356,7 +356,7 @@ abstract class HashAggregateExecTransformer( // The process of handling the inconsistency in column types and order between // Spark and Velox is exactly the opposite of applyExtractStruct. aggregateExpression.mode match { - case PartialMerge | Final => + case PartialMerge | Final | Complete => val newInputAttributes = new ArrayBuffer[Attribute]() val childNodes = new JArrayList[ExpressionNode]() val (sparkOrders, sparkTypes) = @@ -467,7 +467,7 @@ abstract class HashAggregateExecTransformer( // by previous projection. childrenNodes.add(ExpressionBuilder.makeSelection(colIdx)) colIdx += 1 - case Partial => + case Partial | Complete => aggFunc.children.foreach { _ => childrenNodes.add(ExpressionBuilder.makeSelection(colIdx)) @@ -600,7 +600,7 @@ abstract class HashAggregateExecTransformer( } val aggregateFunc = aggExpr.aggregateFunction val childrenNodes = aggExpr.mode match { - case Partial => + case Partial | Complete => aggregateFunc.children.toList.map( expr => { ExpressionConverter @@ -784,7 +784,7 @@ case class HashAggregateExecPullOutHelper( expr.mode match { case Partial | PartialMerge => expr.aggregateFunction.aggBufferAttributes - case Final => + case Final | Complete => Seq(aggregateAttributes(index)) case other => throw new GlutenNotSupportException(s"Unsupported aggregate mode: $other.") diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala index a853778484b1..9cdcf854db8b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/RowToVeloxColumnarExec.scala @@ -17,6 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -43,7 +44,6 @@ import org.apache.arrow.memory.ArrowBuf import scala.collection.mutable.ListBuffer case class RowToVeloxColumnarExec(child: SparkPlan) extends RowToColumnarExecBase(child = child) { - override def doExecuteColumnarInternal(): RDD[ColumnarBatch] = { val numInputRows = longMetric("numInputRows") val numOutputBatches = longMetric("numOutputBatches") @@ -122,7 +122,7 @@ object RowToVeloxColumnarExec { val arrowSchema = SparkArrowUtil.toArrowSchema(schema, SQLConf.get.sessionLocalTimeZone) - val runtime = Runtimes.contextInstance("RowToColumnar") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "RowToColumnar") val jniWrapper = NativeRowToColumnarJniWrapper.create(runtime) val arrowAllocator = ArrowBufferAllocators.contextInstance() val cSchema = ArrowSchema.allocateNew(arrowAllocator) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala index 01c89bee217b..f3adbe351aa4 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/TopNTransformer.scala @@ -79,7 +79,7 @@ case class TopNTransformer( child.output, childCtx.root, validation = false) - TransformContext(child.output, child.output, relNode) + TransformContext(child.output, relNode) } private def getRelNode( diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala index 8aedeb87cb28..4c0f79538c0f 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxColumnarToRowExec.scala @@ -16,6 +16,7 @@ */ package org.apache.gluten.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.extension.ValidationResult @@ -122,7 +123,7 @@ object VeloxColumnarToRowExec { return Iterator.empty } - val runtime = Runtimes.contextInstance("ColumnarToRow") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ColumnarToRow") // TODO: Pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast. val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) val c2rId = jniWrapper.nativeColumnarToRowInit() @@ -156,7 +157,7 @@ object VeloxColumnarToRowExec { val cols = batch.numCols() val rows = batch.numRows() val beforeConvert = System.currentTimeMillis() - val batchHandle = ColumnarBatches.getNativeHandle(batch) + val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) var info = jniWrapper.nativeColumnarToRowConvert(c2rId, batchHandle, 0) diff --git a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala index 995582024ba5..9ed687d33703 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/execution/VeloxResizeBatchesExec.scala @@ -16,7 +16,8 @@ */ package org.apache.gluten.execution -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.iterator.Iterators import org.apache.gluten.utils.VeloxBatchResizer @@ -53,7 +54,10 @@ case class VeloxResizeBatchesExec( "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to append / split batches") ) - override def supportsColumnar: Boolean = true + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala similarity index 97% rename from backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala rename to backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala index b1b0b813f6c8..5e02cf54b0e5 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/datasource/ArrowConvertorRule.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/extension/ArrowConvertorRule.scala @@ -14,9 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.datasource +package org.apache.gluten.extension import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.datasource.ArrowCSVFileFormat import org.apache.gluten.datasource.v2.ArrowCSVTable import org.apache.gluten.sql.shims.SparkShimLoader diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala index a9067d069e03..8002a44ae92b 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/InputIteratorMetricsUpdater.scala @@ -17,20 +17,23 @@ package org.apache.gluten.metrics import org.apache.spark.sql.execution.metric.SQLMetric -case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric]) extends MetricsUpdater { +case class InputIteratorMetricsUpdater(metrics: Map[String, SQLMetric], forBroadcast: Boolean) + extends MetricsUpdater { override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = { if (opMetrics != null) { val operatorMetrics = opMetrics.asInstanceOf[OperatorMetrics] metrics("cpuCount") += operatorMetrics.cpuCount metrics("wallNanos") += operatorMetrics.wallNanos - if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) { - // Sometimes, velox does not update metrics for intermediate operator, - // here we try to use the input metrics - metrics("numOutputRows") += operatorMetrics.inputRows - metrics("outputVectors") += operatorMetrics.inputVectors - } else { - metrics("numOutputRows") += operatorMetrics.outputRows - metrics("outputVectors") += operatorMetrics.outputVectors + if (!forBroadcast) { + if (operatorMetrics.outputRows == 0 && operatorMetrics.outputVectors == 0) { + // Sometimes, velox does not update metrics for intermediate operator, + // here we try to use the input metrics + metrics("numOutputRows") += operatorMetrics.inputRows + metrics("outputVectors") += operatorMetrics.inputVectors + } else { + metrics("numOutputRows") += operatorMetrics.outputRows + metrics("outputVectors") += operatorMetrics.outputVectors + } } } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala index cd50d0b8e20c..b8ef1620f905 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/MetricsUtil.scala @@ -58,7 +58,8 @@ object MetricsUtil extends Logging { assert(t.children.size == 1, "MetricsUpdater.None can only be used on unary operator") treeifyMetricsUpdaters(t.children.head) case t: TransformSupport => - MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) + // Reversed children order to match the traversal code. + MetricsUpdaterTree(t.metricsUpdater(), t.children.reverse.map(treeifyMetricsUpdaters)) case _ => MetricsUpdaterTree(MetricsUpdater.Terminate, Seq()) } @@ -233,6 +234,12 @@ object MetricsUtil extends Logging { operatorMetrics, metrics.getSingleMetrics, joinParamsMap.get(operatorIdx)) + case u: UnionMetricsUpdater => + // JoinRel outputs two suites of metrics respectively for hash build and hash probe. + // Therefore, fetch one more suite of metrics here. + operatorMetrics.add(metrics.getOperatorMetrics(curMetricsIdx)) + curMetricsIdx -= 1 + u.updateUnionMetrics(operatorMetrics) case hau: HashAggregateMetricsUpdater => hau.updateAggregationMetrics(operatorMetrics, aggParamsMap.get(operatorIdx)) case lu: LimitMetricsUpdater => diff --git a/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala new file mode 100644 index 000000000000..9e91cf368c0a --- /dev/null +++ b/backends-velox/src/main/scala/org/apache/gluten/metrics/UnionMetricsUpdater.scala @@ -0,0 +1,35 @@ +/* + * 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.gluten.metrics + +import org.apache.spark.sql.execution.metric.SQLMetric + +class UnionMetricsUpdater(val metrics: Map[String, SQLMetric]) extends MetricsUpdater { + override def updateNativeMetrics(opMetrics: IOperatorMetrics): Unit = { + throw new UnsupportedOperationException() + } + + def updateUnionMetrics(unionMetrics: java.util.ArrayList[OperatorMetrics]): Unit = { + // Union was interpreted to LocalExchange + LocalPartition. Use metrics from LocalExchange. + val localExchangeMetrics = unionMetrics.get(0) + metrics("numInputRows") += localExchangeMetrics.inputRows + metrics("inputVectors") += localExchangeMetrics.inputVectors + metrics("inputBytes") += localExchangeMetrics.inputBytes + metrics("cpuCount") += localExchangeMetrics.cpuCount + metrics("wallNanos") += localExchangeMetrics.wallNanos + } +} diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala index 16cd18e41a05..b9d90d589c02 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos7.scala @@ -37,7 +37,6 @@ class SharedLibraryLoaderCentos7 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.10", "libre2.so", false) loader.loadAndCreateLink("libzstd.so.1", "libzstd.so", false) loader.loadAndCreateLink("liblz4.so.1", "liblz4.so", false) diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala index 0a75c30c22ff..dbb7d59f889d 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos8.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos8 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.0", "libre2.so", false) loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala index 50f9fe4aaadc..f633a79f8fcd 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderCentos9.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderCentos9 extends SharedLibraryLoader { loader.loadAndCreateLink("libntlm.so.0", "libntlm.so", false) loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.9", "libre2.so", false) loader.loadAndCreateLink("libsodium.so.23", "libsodium.so", false) } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala index 06c065ba2883..4f9b6fdd7250 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian11.scala @@ -44,6 +44,5 @@ class SharedLibraryLoaderDebian11 extends SharedLibraryLoader { loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala index 8018995328fe..e3967eea3267 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderDebian12.scala @@ -50,6 +50,5 @@ class SharedLibraryLoaderDebian12 extends SharedLibraryLoader { loader.loadAndCreateLink("libevent-2.1.so.7", "libevent-2.1.so", false) loader.loadAndCreateLink("libcurl.so.4", "libcurl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) } } diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala index d1f21a0013fb..6d9271e9e3d9 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2004.scala @@ -57,7 +57,6 @@ class SharedLibraryLoaderUbuntu2004 extends SharedLibraryLoader { loader.loadAndCreateLink("libicudata.so.66", "libicudata.so", false) loader.loadAndCreateLink("libicuuc.so.66", "libicuuc.so", false) loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.5", "libre2.so", false) loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libthrift-0.13.0.so", "libthrift.so", false) diff --git a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala index 3cf4d30237ac..95f7db5655b2 100755 --- a/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/utils/SharedLibraryLoaderUbuntu2204.scala @@ -42,7 +42,6 @@ class SharedLibraryLoaderUbuntu2204 extends SharedLibraryLoader { loader.loadAndCreateLink("libgsasl.so.7", "libgsasl.so", false) loader.loadAndCreateLink("libprotobuf.so.32", "libprotobuf.so", false) loader.loadAndCreateLink("libxml2.so.2", "libxml2.so", false) - loader.loadAndCreateLink("libhdfs.so.0.0.0", "libhdfs.so", false) loader.loadAndCreateLink("libre2.so.9", "libre2.so", false) loader.loadAndCreateLink("libsnappy.so.1", "libsnappy.so", false) loader.loadAndCreateLink("libthrift-0.16.0.so", "libthrift.so", false) diff --git a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala index fa4d92652ca3..cd035e3202d2 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/vectorized/ColumnarBatchSerializer.scala @@ -17,6 +17,7 @@ package org.apache.gluten.vectorized import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.iterator.ClosableIterator import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes @@ -98,13 +99,15 @@ private class ColumnarBatchSerializerInstance( val compressionCodecBackend = GlutenConfig.getConf.columnarShuffleCodecBackend.orNull val batchSize = GlutenConfig.getConf.maxBatchSize - val runtime = Runtimes.contextInstance("ShuffleReader") + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader") val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val shuffleReaderHandle = jniWrapper.make( cSchema.memoryAddress(), compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType) // Close shuffle reader instance as lately as the end of task processing, // since the native reader could hold a reference to memory pool that @@ -133,7 +136,8 @@ private class ColumnarBatchSerializerInstance( extends DeserializationStream with TaskResource { private val byteIn: JniByteInputStream = JniByteInputStreams.create(in) - private val runtime = Runtimes.contextInstance("ShuffleReader") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleReader") private val wrappedOut: ClosableIterator = new ColumnarBatchOutIterator( runtime, ShuffleReaderJniWrapper diff --git a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala index f1f5eb906238..4a5f43b7fdfa 100644 --- a/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/api/python/ColumnarArrowEvalPythonExec.scala @@ -19,9 +19,8 @@ package org.apache.spark.api.python import org.apache.gluten.columnarbatch.ArrowBatches.ArrowJavaBatch import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.exception.GlutenException -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.utils.PullOutProjectHelper @@ -212,14 +211,14 @@ case class ColumnarArrowEvalPythonExec( child: SparkPlan, evalType: Int) extends EvalPythonExec - with GlutenPlan - with KnownChildrenConventions { - override def supportsColumnar: Boolean = true + with GlutenPlan { - override protected def batchType0(): Convention.BatchType = ArrowJavaBatch + override def batchType(): Convention.BatchType = ArrowJavaBatch - override def requiredChildrenConventions(): Seq[ConventionReq] = List( - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(ArrowJavaBatch))) + override def rowType0(): Convention.RowType = Convention.RowType.None + + override def requiredChildConvention(): Seq[ConventionReq] = List( + ConventionReq.ofBatch(ConventionReq.BatchType.Is(ArrowJavaBatch))) override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), diff --git a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala index eaf9d99a9ecc..e9f821512e16 100644 --- a/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala +++ b/backends-velox/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} import org.apache.gluten.runtime.Runtimes @@ -99,7 +100,7 @@ class ColumnarShuffleWriter[K, V]( private val reallocThreshold = GlutenConfig.getConf.columnarShuffleReallocThreshold - private val runtime = Runtimes.contextInstance("ShuffleWriter") + private val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "ShuffleWriter") private val jniWrapper = ShuffleWriterJniWrapper.create(runtime) @@ -135,7 +136,7 @@ class ColumnarShuffleWriter[K, V]( logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") } else { val rows = cb.numRows() - val handle = ColumnarBatches.getNativeHandle(cb) + val handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, cb) if (nativeShuffleWriter == -1L) { nativeShuffleWriter = jniWrapper.make( dep.nativePartitioning.getShortName, diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala index 85bc68223487..16b8fb0e9f6f 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ArrowFileSourceScanExec.scala @@ -33,8 +33,6 @@ case class ArrowFileSourceScanExec(original: FileSourceScanExec) override def output: Seq[Attribute] = original.output - override def supportsColumnar: Boolean = original.supportsColumnar - override def doCanonicalize(): FileSourceScanExec = original.doCanonicalize() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala index 38a6d1803d0b..1aacc1b95416 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BaseArrowScanExec.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution import org.apache.gluten.columnarbatch.ArrowBatches -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.transition.Convention trait BaseArrowScanExec extends GlutenPlan { - - final override protected def batchType0(): Convention.BatchType = { + final override def batchType(): Convention.BatchType = { ArrowBatches.ArrowJavaBatch } + + final override def rowType0(): Convention.RowType = Convention.RowType.None } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala index 5bc6b7c56da5..c5323d4f8d50 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/BroadcastUtils.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.runtime.Runtimes import org.apache.gluten.sql.shims.SparkShimLoader @@ -105,7 +106,8 @@ object BroadcastUtils { } ColumnarBuildSideRelation( SparkShimLoader.getSparkShims.attributesFromStruct(schema), - serialized) + serialized, + mode) } // Rebroadcast Velox relation. context.broadcast(toRelation).asInstanceOf[Broadcast[T]] @@ -123,7 +125,8 @@ object BroadcastUtils { } ColumnarBuildSideRelation( SparkShimLoader.getSparkShims.attributesFromStruct(schema), - serialized) + serialized, + mode) } // Rebroadcast Velox relation. context.broadcast(toRelation).asInstanceOf[Broadcast[T]] @@ -152,11 +155,14 @@ object BroadcastUtils { if (filtered.isEmpty) { return ColumnarBatchSerializeResult.EMPTY } - val handleArray = filtered.map(ColumnarBatches.getNativeHandle) + val handleArray = + filtered.map(b => ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, b)) val serializeResult = try { ColumnarBatchSerializerJniWrapper - .create(Runtimes.contextInstance("BroadcastUtils#serializeStream")) + .create( + Runtimes + .contextInstance(BackendsApiManager.getBackendName, "BroadcastUtils#serializeStream")) .serialize(handleArray) } finally { filtered.foreach(ColumnarBatches.release) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala index feaf72f64fb2..977357990c43 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -26,7 +27,10 @@ import org.apache.gluten.vectorized.{ColumnarBatchSerializerJniWrapper, NativeCo import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode +import org.apache.spark.sql.catalyst.plans.physical.IdentityBroadcastMode import org.apache.spark.sql.execution.joins.BuildSideRelation +import org.apache.spark.sql.execution.joins.HashedRelationBroadcastMode import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.utils.SparkArrowUtil import org.apache.spark.sql.vectorized.ColumnarBatch @@ -36,11 +40,22 @@ import org.apache.arrow.c.ArrowSchema import scala.collection.JavaConverters.asScalaIteratorConverter -case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Array[Byte]]) +case class ColumnarBuildSideRelation( + output: Seq[Attribute], + batches: Array[Array[Byte]], + mode: BroadcastMode) extends BuildSideRelation { + private def transformProjection: UnsafeProjection = { + mode match { + case HashedRelationBroadcastMode(k, _) => UnsafeProjection.create(k) + case IdentityBroadcastMode => UnsafeProjection.create(output, output) + } + } + override def deserialized: Iterator[ColumnarBatch] = { - val runtime = Runtimes.contextInstance("BuildSideRelation#deserialized") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#deserialized") val jniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime) val serializeHandle: Long = { val allocator = ArrowBufferAllocators.contextInstance() @@ -82,11 +97,15 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra override def asReadOnlyCopy(): ColumnarBuildSideRelation = this /** - * Transform columnar broadcast value to Array[InternalRow] by key and distinct. NOTE: This method - * was called in Spark Driver, should manage resources carefully. + * Transform columnar broadcast value to Array[InternalRow] by key. + * + * NOTE: + * - This method was called in Spark Driver, should manage resources carefully. + * - The "key" must be already been bound reference. */ override def transform(key: Expression): Array[InternalRow] = TaskResources.runUnsafe { - val runtime = Runtimes.contextInstance("BuildSideRelation#transform") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "BuildSideRelation#transform") // This transformation happens in Spark driver, thus resources can not be managed automatically. val serializerJniWrapper = ColumnarBatchSerializerJniWrapper.create(runtime) val serializeHandle = { @@ -103,6 +122,8 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra var closed = false + val proj = UnsafeProjection.create(Seq(key)) + // Convert columnar to Row. val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) val c2rId = jniWrapper.nativeColumnarToRowInit() @@ -138,10 +159,9 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra var info = jniWrapper.nativeColumnarToRowConvert( c2rId, - ColumnarBatches.getNativeHandle(batch), + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch), 0) batch.close() - val proj = UnsafeProjection.create(Seq(key), output) new Iterator[InternalRow] { var rowId = 0 @@ -164,7 +184,7 @@ case class ColumnarBuildSideRelation(output: Seq[Attribute], batches: Array[Arra rowId += 1 row } - }.map(proj).map(_.copy()) + }.map(transformProjection).map(proj).map(_.copy()) } } } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala index d6e7aa5b2244..64ad105c7f7c 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala @@ -46,7 +46,7 @@ case class CachedColumnarBatch( bytes: Array[Byte]) extends CachedBatch {} -// spotless:off +// format: off /** * Feature: * 1. This serializer supports column pruning @@ -75,7 +75,7 @@ case class CachedColumnarBatch( * - Deserializer DefaultCachedBatch -> InternalRow (unsupport ColumnarToRow) * -> Convert DefaultCachedBatch to InternalRow using vanilla Spark serializer */ -// spotless:on +// format: on class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHelper with Logging { private lazy val rowBasedCachedBatchSerializer = new DefaultCachedBatchSerializer @@ -176,8 +176,12 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val batch = it.next() val results = ColumnarBatchSerializerJniWrapper - .create(Runtimes.contextInstance("ColumnarCachedBatchSerializer#serialize")) - .serialize(Array(ColumnarBatches.getNativeHandle(batch))) + .create( + Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#serialize")) + .serialize( + Array(ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch))) CachedColumnarBatch( results.getNumRows.toInt, results.getSerialized.length, @@ -201,7 +205,9 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val timezoneId = SQLConf.get.sessionLocalTimeZone input.mapPartitions { it => - val runtime = Runtimes.contextInstance("ColumnarCachedBatchSerializer#read") + val runtime = Runtimes.contextInstance( + BackendsApiManager.getBackendName, + "ColumnarCachedBatchSerializer#read") val jniWrapper = ColumnarBatchSerializerJniWrapper .create(runtime) val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId) @@ -224,7 +230,10 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val batch = ColumnarBatches.create(batchHandle) if (shouldSelectAttributes) { try { - ColumnarBatches.select(batch, requestedColumnIndices.toArray) + ColumnarBatches.select( + BackendsApiManager.getBackendName, + batch, + requestedColumnIndices.toArray) } finally { batch.close() } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala index cd5f442bc765..c9651557fa70 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxFormatWriterInjects.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.velox +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.datasource.{VeloxDataSourceJniWrapper, VeloxDataSourceUtil} import org.apache.gluten.exception.GlutenException @@ -57,7 +58,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { SparkArrowUtil.toArrowSchema(dataSchema, SQLConf.get.sessionLocalTimeZone) val cSchema = ArrowSchema.allocateNew(ArrowBufferAllocators.contextInstance()) var dsHandle = -1L - val runtime = Runtimes.contextInstance("VeloxWriter") + val runtime = Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val allocator = ArrowBufferAllocators.contextInstance() try { @@ -77,7 +78,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { ColumnarBatches.retain(batch) val batchHandle = { ColumnarBatches.checkOffloaded(batch) - ColumnarBatches.getNativeHandle(batch) + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) } datasourceJniWrapper.writeBatch(dsHandle, batchHandle) batch.close() @@ -108,8 +109,9 @@ class VeloxRowSplitter extends GlutenRowSplitter { partitionColIndice: Array[Int], hasBucket: Boolean, reserve_partition_columns: Boolean = false): BlockStripes = { - val handler = ColumnarBatches.getNativeHandle(row.batch) - val runtime = Runtimes.contextInstance("VeloxPartitionWriter") + val handler = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, row.batch) + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "VeloxPartitionWriter") val datasourceJniWrapper = VeloxDataSourceJniWrapper.create(runtime) val originalColumns: Array[Int] = Array.range(0, row.batch.numCols()) val dataColIndice = originalColumns.filterNot(partitionColIndice.contains(_)) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala index 32bac0204519..e3a84d4f074d 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.execution.utils +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.{ColumnarBatches, VeloxColumnarBatches} import org.apache.gluten.iterator.Iterators import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators @@ -41,10 +42,11 @@ import org.apache.spark.util.MutablePair object ExecUtil { def convertColumnarToRow(batch: ColumnarBatch): Iterator[InternalRow] = { - val runtime = Runtimes.contextInstance("ExecUtil#ColumnarToRow") + val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "ExecUtil#ColumnarToRow") val jniWrapper = NativeColumnarToRowJniWrapper.create(runtime) var info: NativeColumnarToRowInfo = null - val batchHandle = ColumnarBatches.getNativeHandle(batch) + val batchHandle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, batch) val c2rHandle = jniWrapper.nativeColumnarToRowInit() info = jniWrapper.nativeColumnarToRowConvert(c2rHandle, batchHandle, 0) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala index d895faa31702..b3524e20f019 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/hive/VeloxHiveUDFTransformer.scala @@ -37,11 +37,11 @@ object VeloxHiveUDFTransformer { } if (UDFResolver.UDFNames.contains(udfClassName)) { - UDFResolver + val udfExpression = UDFResolver .getUdfExpression(udfClassName, udfName)(expr.children) - .getTransformer( - ExpressionConverter.replaceWithExpressionTransformer(expr.children, attributeSeq) - ) + udfExpression.getTransformer( + ExpressionConverter.replaceWithExpressionTransformer(udfExpression.children, attributeSeq) + ) } else { HiveUDFTransformer.genTransformerFromUDFMappings(udfName, expr, attributeSeq) } diff --git a/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala new file mode 100644 index 000000000000..1573042884b2 --- /dev/null +++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala @@ -0,0 +1,20 @@ +/* + * 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.gluten.execution + +class VeloxIcebergSuite extends IcebergSuite diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala similarity index 96% rename from gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala rename to backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala index 857cbcb410f8..5bc26e81b7ca 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala +++ b/backends-velox/src/test-iceberg/scala/org/apache/gluten/execution/VeloxTPCHIcebergSuite.scala @@ -24,13 +24,13 @@ import org.apache.iceberg.spark.SparkWriteOptions import java.io.File class VeloxTPCHIcebergSuite extends VeloxTPCHSuite { - - protected val tpchBasePath: String = new File( - "../backends-velox/src/test/resources").getAbsolutePath + protected val tpchBasePath: String = + getClass.getResource("/").getPath + "../../../src/test/resources" override protected val resourcePath: String = new File(tpchBasePath, "tpch-data-parquet").getCanonicalPath + // FIXME: Unused. override protected val queriesResults: String = new File(tpchBasePath, "queries-output").getCanonicalPath diff --git a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java index 54803aa1930f..f02caf8f2d1b 100644 --- a/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java +++ b/backends-velox/src/test/java/org/apache/gluten/columnarbatch/ColumnarBatchTest.java @@ -16,6 +16,7 @@ */ package org.apache.gluten.columnarbatch; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.execution.RowToVeloxColumnarExec; import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators; import org.apache.gluten.test.VeloxBackendTestBase; @@ -114,9 +115,12 @@ public void testCreateByHandle() { final ColumnarBatch offloaded = ColumnarBatches.offload(ArrowBufferAllocators.contextInstance(), batch); Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded)); - final long handle = ColumnarBatches.getNativeHandle(offloaded); + final long handle = + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), offloaded); final ColumnarBatch created = ColumnarBatches.create(handle); - Assert.assertEquals(handle, ColumnarBatches.getNativeHandle(created)); + Assert.assertEquals( + handle, + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), created)); Assert.assertEquals(1, ColumnarBatches.getRefCnt(offloaded)); Assert.assertEquals(1, ColumnarBatches.getRefCnt(created)); ColumnarBatches.retain(created); diff --git a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java index b5ce30e39b25..254e8e2830f4 100644 --- a/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java +++ b/backends-velox/src/test/java/org/apache/gluten/test/MockVeloxBackend.java @@ -71,6 +71,7 @@ public Object ask(Object message) throws Exception { private static SparkConf newSparkConf() { final SparkConf conf = new SparkConf(); conf.set(GlutenConfig.SPARK_OFFHEAP_SIZE_KEY(), "1g"); + conf.set(GlutenConfig.COLUMNAR_VELOX_CONNECTOR_IO_THREADS(), "0"); return conf; } } diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt index 5b913a06db13..7197e83986a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt index 946c150d8654..4687bd81bcc8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt index 39b1c80d441b..cd765112a70f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (46) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (46) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt index f7b9f730e38b..10972d6c2f1a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt index 2cd26d020ae7..88d48cf5dfa3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -324,7 +324,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt index 2ef6280501aa..a1adaf31bf28 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/21.txt @@ -20,34 +20,34 @@ AdaptiveSparkPlan (93) : : : +- BroadcastQueryStage (5) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (92) +- HashAggregate (91) @@ -83,7 +83,7 @@ AdaptiveSparkPlan (93) +- Scan parquet (83) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -203,7 +203,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -245,7 +245,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt index 271e9e810ec3..afb719f19903 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt index f0fca318bf20..9335e4fbe92c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt index b8cdf1656c05..9ff9aba0a986 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt index 0ef8d9d4fc1a..df5fa26f0098 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt index cac53895b981..88f83c467e85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt index 33544f0796a5..c389ab06a45f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt index 7112bb021b0c..830259ba82bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark32/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt index 42cf8e1b7f58..c15913c284f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt index 8956ddaa2184..b06cd55156e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt index d7603e49ffb6..52a87f43535d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -237,34 +237,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt index 394bbd2469c2..8cc2b417eb3f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt index 6c279f2d124d..8fd71a81a264 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -323,7 +323,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt index eae4758613ee..505dc59dc4cb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -202,7 +202,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt index a3d3f0a08eeb..9113355acbdd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -227,7 +227,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -237,7 +237,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -344,7 +344,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt index e7fa21b6f1f1..befea454305b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt index a1eaaca4d448..52c99692cf4d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt index 638a442362e3..a076ea86feb5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt index 86db01e58b53..5fdae3d7334b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt index ccc7941882b7..0b00093c0316 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt index 212413568214..8c1db077485f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark33/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt index 47dc04d100db..a4a335e80831 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt index 6d2233b3e09f..61b9acc60dc0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt index 1ec03a2f9edd..48b06986ee1b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,34 +239,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt index e64cb0990026..3a9b17afd60c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt index c92587c9d791..c3ab86e5acfb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(5) Scan parquet +(5) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -288,7 +288,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt index 8382dbcd504e..5fab9fa9da42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -205,7 +205,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -248,7 +248,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt index 7f3836740155..0c2ce3c56b5a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -229,7 +229,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -239,7 +239,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,7 +346,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt index 1035e0eb7715..cc68e1c9b249 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -104,7 +104,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt index 150788a3d7c1..a77b48298d37 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt index 035c5c388972..58442a8ca86e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -144,7 +144,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -183,7 +183,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -222,7 +222,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt index cd877f54bd7a..eaa84cfa46c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,7 +134,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -212,7 +212,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt index d381d21da73d..9d1bc4a17558 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -169,7 +169,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -208,7 +208,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -247,7 +247,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -286,7 +286,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -325,7 +325,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -364,7 +364,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt index f0ebba6efac5..8baec4075ad3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj-ras/spark34/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -220,7 +220,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,7 +259,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt index 5b913a06db13..7197e83986a8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt index 946c150d8654..4687bd81bcc8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt index 39b1c80d441b..cd765112a70f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/15.txt @@ -13,7 +13,7 @@ AdaptiveSparkPlan (46) : +- BroadcastQueryStage (5) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -24,7 +24,7 @@ AdaptiveSparkPlan (46) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -71,7 +71,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt index f7b9f730e38b..10972d6c2f1a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt index 2cd26d020ae7..88d48cf5dfa3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -207,7 +207,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -324,7 +324,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt index 2ef6280501aa..a1adaf31bf28 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/21.txt @@ -20,34 +20,34 @@ AdaptiveSparkPlan (93) : : : +- BroadcastQueryStage (5) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (92) +- HashAggregate (91) @@ -83,7 +83,7 @@ AdaptiveSparkPlan (93) +- Scan parquet (83) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -112,7 +112,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -127,7 +127,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -203,7 +203,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -245,7 +245,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt index 271e9e810ec3..afb719f19903 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt index f0fca318bf20..9335e4fbe92c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt index b8cdf1656c05..9ff9aba0a986 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt index 0ef8d9d4fc1a..df5fa26f0098 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt index cac53895b981..88f83c467e85 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt index 33544f0796a5..c389ab06a45f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt index 7112bb021b0c..830259ba82bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark32/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt index 42cf8e1b7f58..c15913c284f5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt index 8956ddaa2184..b06cd55156e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt index d7603e49ffb6..52a87f43535d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -237,34 +237,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -326,71 +325,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt index 394bbd2469c2..8cc2b417eb3f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt index 6c279f2d124d..8fd71a81a264 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (26) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -323,7 +323,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(55) Scan parquet +(55) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt index eae4758613ee..505dc59dc4cb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -155,7 +155,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -202,7 +202,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt index a3d3f0a08eeb..9113355acbdd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -227,7 +227,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -237,7 +237,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -344,7 +344,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt index e7fa21b6f1f1..befea454305b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -103,7 +103,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt index a1eaaca4d448..52c99692cf4d 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt index 638a442362e3..a076ea86feb5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +143,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -219,7 +219,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,7 +257,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt index 86db01e58b53..5fdae3d7334b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,7 +133,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -209,7 +209,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt index ccc7941882b7..0b00093c0316 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -206,7 +206,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -244,7 +244,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -320,7 +320,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -358,7 +358,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt index 212413568214..8c1db077485f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark33/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -141,7 +141,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -179,7 +179,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,7 +217,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -255,7 +255,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt index 47dc04d100db..a4a335e80831 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/10.txt @@ -18,24 +18,24 @@ AdaptiveSparkPlan (68) : :- ^ ProjectExecTransformer (12) : : +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) : : :- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (10) : : +- BroadcastQueryStage (8), Statistics(X) : : +- ColumnarBroadcastExchange (7) : : +- ^ ProjectExecTransformer (5) : : +- ^ FilterExecTransformer (4) - : : +- ^ Scan parquet (3) + : : +- ^ ScanTransformer parquet (3) : +- ^ InputIteratorTransformer (20) : +- BroadcastQueryStage (18), Statistics(X) : +- ColumnarBroadcastExchange (17) : +- ^ ProjectExecTransformer (15) : +- ^ FilterExecTransformer (14) - : +- ^ Scan parquet (13) + : +- ^ ScanTransformer parquet (13) +- ^ InputIteratorTransformer (29) +- BroadcastQueryStage (27), Statistics(X) +- ColumnarBroadcastExchange (26) +- ^ FilterExecTransformer (24) - +- ^ Scan parquet (23) + +- ^ ScanTransformer parquet (23) +- == Initial Plan == TakeOrderedAndProject (67) +- HashAggregate (66) @@ -62,7 +62,7 @@ AdaptiveSparkPlan (68) +- Scan parquet (59) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +73,7 @@ ReadSchema: struct -(2) Scan parquet +(2) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt index 6d2233b3e09f..61b9acc60dc0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/14.txt @@ -13,12 +13,12 @@ AdaptiveSparkPlan (35) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (11) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (10) +- BroadcastQueryStage (8), Statistics(X) +- ColumnarBroadcastExchange (7) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == HashAggregate (34) +- Exchange (33) @@ -33,7 +33,7 @@ AdaptiveSparkPlan (35) +- Scan parquet (27) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -48,7 +48,7 @@ Arguments: (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_s Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt index 1ec03a2f9edd..48b06986ee1b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/15.txt @@ -12,7 +12,7 @@ AdaptiveSparkPlan (43) : +- BroadcastQueryStage (5), Statistics(X) : +- ColumnarBroadcastExchange (4) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (20) +- ^ RegularHashAggregateExecTransformer (19) +- ^ InputIteratorTransformer (18) @@ -23,7 +23,7 @@ AdaptiveSparkPlan (43) +- ^ FlushableHashAggregateExecTransformer (11) +- ^ ProjectExecTransformer (10) +- ^ FilterExecTransformer (9) - +- ^ Scan parquet (8) + +- ^ ScanTransformer parquet (8) +- == Initial Plan == Sort (42) +- Exchange (41) @@ -41,7 +41,7 @@ AdaptiveSparkPlan (43) +- Scan parquet (32) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -70,7 +70,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (7) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,34 +239,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 20 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (69) +AdaptiveSparkPlan (68) +- == Final Plan == - VeloxColumnarToRow (60) - +- ^ RegularHashAggregateExecTransformer (58) - +- ^ RegularHashAggregateExecTransformer (57) - +- ^ ProjectExecTransformer (56) - +- ^ RegularHashAggregateExecTransformer (55) - +- ^ InputIteratorTransformer (54) - +- ShuffleQueryStage (52), Statistics(X) - +- ColumnarExchange (51) - +- VeloxResizeBatches (50) - +- ^ ProjectExecTransformer (48) - +- ^ FlushableHashAggregateExecTransformer (47) - +- ^ ProjectExecTransformer (46) - +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + VeloxColumnarToRow (59) + +- ^ RegularHashAggregateExecTransformer (57) + +- ^ ProjectExecTransformer (56) + +- ^ RegularHashAggregateExecTransformer (55) + +- ^ InputIteratorTransformer (54) + +- ShuffleQueryStage (52), Statistics(X) + +- ColumnarExchange (51) + +- VeloxResizeBatches (50) + +- ^ ProjectExecTransformer (48) + +- ^ FlushableHashAggregateExecTransformer (47) + +- ^ ProjectExecTransformer (46) + +- ^ FilterExecTransformer (45) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == - HashAggregate (68) - +- HashAggregate (67) - +- HashAggregate (66) - +- Exchange (65) - +- HashAggregate (64) - +- Project (63) - +- Filter (62) - +- Scan parquet (61) + HashAggregate (67) + +- HashAggregate (66) + +- HashAggregate (65) + +- Exchange (64) + +- HashAggregate (63) + +- Project (62) + +- Filter (61) + +- Scan parquet (60) -(44) Scan parquet +(44) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -328,71 +327,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (57) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(58) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(59) WholeStageCodegenTransformer (X) +(58) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(60) VeloxColumnarToRow +(59) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(61) Scan parquet +(60) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(62) Filter +(61) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(63) Project +(62) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(64) HashAggregate +(63) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(65) Exchange +(64) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(66) HashAggregate +(65) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(67) HashAggregate +(66) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(68) HashAggregate +(67) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(69) AdaptiveSparkPlan +(68) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt index e64cb0990026..3a9b17afd60c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/16.txt @@ -24,12 +24,12 @@ AdaptiveSparkPlan (59) +- ^ ProjectExecTransformer (11) +- ^ BroadcastHashJoinExecTransformer Inner BuildRight (10) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (9) +- BroadcastQueryStage (7), Statistics(X) +- ColumnarBroadcastExchange (6) +- ^ FilterExecTransformer (4) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (58) +- Exchange (57) @@ -53,7 +53,7 @@ AdaptiveSparkPlan (59) +- Scan parquet (46) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -64,7 +64,7 @@ ReadSchema: struct Input [2]: [ps_partkey#X, ps_suppkey#X] Arguments: isnotnull(ps_partkey#X) -(3) Scan parquet +(3) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt index c92587c9d791..c3ab86e5acfb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/18.txt @@ -18,10 +18,10 @@ AdaptiveSparkPlan (88) : : +- BroadcastQueryStage (5), Statistics(X) : : +- ColumnarBroadcastExchange (4) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (27) : :- ^ FilterExecTransformer (9) - : : +- ^ Scan parquet (8) + : : +- ^ ScanTransformer parquet (8) : +- ^ InputIteratorTransformer (26) : +- BroadcastQueryStage (24), Statistics(X) : +- ColumnarBroadcastExchange (23) @@ -34,13 +34,13 @@ AdaptiveSparkPlan (88) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FlushableHashAggregateExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (41) +- BroadcastQueryStage (39), Statistics(X) +- ColumnarBroadcastExchange (38) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (36) :- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (35) +- BroadcastQueryStage (33), Statistics(X) +- ReusedExchange (32) @@ -79,7 +79,7 @@ AdaptiveSparkPlan (88) +- Scan parquet (73) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -108,7 +108,7 @@ Input [2]: [c_custkey#X, c_name#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ ReadSchema: struct Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(5) Scan parquet +(5) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -168,7 +168,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -288,7 +288,7 @@ Join condition: None Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt index 8382dbcd504e..5fab9fa9da42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/21.txt @@ -19,34 +19,34 @@ AdaptiveSparkPlan (92) : : : +- BroadcastQueryStage (5), Statistics(X) : : : +- ColumnarBroadcastExchange (4) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (26) : : :- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (17) : : : :- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (16) : : : +- BroadcastQueryStage (14), Statistics(X) : : : +- ColumnarBroadcastExchange (13) - : : : +- ^ Scan parquet (11) + : : : +- ^ ScanTransformer parquet (11) : : +- ^ InputIteratorTransformer (25) : : +- BroadcastQueryStage (23), Statistics(X) : : +- ColumnarBroadcastExchange (22) : : +- ^ ProjectExecTransformer (20) : : +- ^ FilterExecTransformer (19) - : : +- ^ Scan parquet (18) + : : +- ^ ScanTransformer parquet (18) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ ProjectExecTransformer (31) : +- ^ FilterExecTransformer (30) - : +- ^ Scan parquet (29) + : +- ^ ScanTransformer parquet (29) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ ProjectExecTransformer (41) +- ^ FilterExecTransformer (40) - +- ^ Scan parquet (39) + +- ^ ScanTransformer parquet (39) +- == Initial Plan == TakeOrderedAndProject (91) +- HashAggregate (90) @@ -82,7 +82,7 @@ AdaptiveSparkPlan (92) +- Scan parquet (82) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -111,7 +111,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (7) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -126,7 +126,7 @@ Arguments: ((((isnotnull(l_receiptdate#X) AND isnotnull(l_commitdate#X)) AND (l_ Output [2]: [l_orderkey#X, l_suppkey#X] Input [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] -(11) Scan parquet +(11) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(18) Scan parquet +(18) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -205,7 +205,7 @@ Join condition: None Output [3]: [s_name#X, s_nationkey#X, l_orderkey#X] Input [5]: [s_suppkey#X, s_name#X, s_nationkey#X, l_orderkey#X, l_suppkey#X] -(29) Scan parquet +(29) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -248,7 +248,7 @@ Join condition: None Output [2]: [s_name#X, s_nationkey#X] Input [4]: [s_name#X, s_nationkey#X, l_orderkey#X, o_orderkey#X] -(39) Scan parquet +(39) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt index 7f3836740155..0c2ce3c56b5a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/22.txt @@ -17,11 +17,11 @@ AdaptiveSparkPlan (40) +- ^ ProjectExecTransformer (10) +- ^ BroadcastHashJoinExecTransformer LeftAnti BuildRight (9) :- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (8) +- BroadcastQueryStage (6), Statistics(X) +- ColumnarBroadcastExchange (5) - +- ^ Scan parquet (3) + +- ^ ScanTransformer parquet (3) +- == Initial Plan == Sort (39) +- Exchange (38) @@ -36,7 +36,7 @@ AdaptiveSparkPlan (40) +- Scan parquet (31) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -47,7 +47,7 @@ ReadSchema: struct Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Arguments: ((isnotnull(c_acctbal#X) AND substring(c_phone#X, 1, 2) IN (13,31,23,29,30,18,17)) AND (cast(c_acctbal#X as decimal(16,6)) > Subquery subquery#X, [id=#X])) -(3) Scan parquet +(3) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -229,7 +229,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) @@ -239,7 +239,7 @@ AdaptiveSparkPlan (60) +- Scan parquet (54) -(41) Scan parquet +(41) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,7 +346,7 @@ AdaptiveSparkPlan (60) +- ^ FlushableHashAggregateExecTransformer (44) +- ^ ProjectExecTransformer (43) +- ^ FilterExecTransformer (42) - +- ^ Scan parquet (41) + +- ^ ScanTransformer parquet (41) +- == Initial Plan == HashAggregate (59) +- Exchange (58) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt index 1035e0eb7715..cc68e1c9b249 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/3.txt @@ -20,15 +20,15 @@ AdaptiveSparkPlan (54) : : +- ColumnarBroadcastExchange (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ FilterExecTransformer (10) - : +- ^ Scan parquet (9) + : +- ^ ScanTransformer parquet (9) +- ^ InputIteratorTransformer (20) +- BroadcastQueryStage (18), Statistics(X) +- ColumnarBroadcastExchange (17) +- ^ ProjectExecTransformer (15) +- ^ FilterExecTransformer (14) - +- ^ Scan parquet (13) + +- ^ ScanTransformer parquet (13) +- == Initial Plan == TakeOrderedAndProject (53) +- HashAggregate (52) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (54) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -104,7 +104,7 @@ Join condition: None Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt index 150788a3d7c1..a77b48298d37 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/4.txt @@ -18,13 +18,13 @@ AdaptiveSparkPlan (46) +- ^ BroadcastHashJoinExecTransformer LeftSemi BuildRight (12) :- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (11) +- BroadcastQueryStage (9), Statistics(X) +- ColumnarBroadcastExchange (8) +- ^ ProjectExecTransformer (6) +- ^ FilterExecTransformer (5) - +- ^ Scan parquet (4) + +- ^ ScanTransformer parquet (4) +- == Initial Plan == Sort (45) +- Exchange (44) @@ -42,7 +42,7 @@ AdaptiveSparkPlan (46) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -57,7 +57,7 @@ Arguments: ((isnotnull(o_orderdate#X) AND (o_orderdate#X >= 1993-07-01)) AND (o_ Output [2]: [o_orderkey#X, o_orderpriority#X] Input [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] -(4) Scan parquet +(4) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt index 035c5c388972..58442a8ca86e 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/5.txt @@ -28,31 +28,31 @@ AdaptiveSparkPlan (102) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ ProjectExecTransformer (10) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (47) +- BroadcastQueryStage (45), Statistics(X) +- ColumnarBroadcastExchange (44) +- ^ ProjectExecTransformer (42) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (101) +- Exchange (100) @@ -90,7 +90,7 @@ AdaptiveSparkPlan (102) +- Scan parquet (91) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -119,7 +119,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (7) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -144,7 +144,7 @@ Join condition: None Output [2]: [c_nationkey#X, o_orderkey#X] Input [4]: [c_custkey#X, c_nationkey#X, o_orderkey#X, o_custkey#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -183,7 +183,7 @@ Join condition: None Output [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [c_nationkey#X, o_orderkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -222,7 +222,7 @@ Join condition: None Output [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Join condition: None Output [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] Input [6]: [l_extendedprice#X, l_discount#X, s_nationkey#X, n_nationkey#X, n_name#X, n_regionkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt index cd877f54bd7a..eaa84cfa46c2 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/7.txt @@ -28,24 +28,24 @@ AdaptiveSparkPlan (95) : : : : : +- BroadcastQueryStage (5), Statistics(X) : : : : : +- ColumnarBroadcastExchange (4) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (9) - : : : : +- ^ Scan parquet (8) + : : : : +- ^ ScanTransformer parquet (8) : : : +- ^ InputIteratorTransformer (18) : : : +- BroadcastQueryStage (16), Statistics(X) : : : +- ColumnarBroadcastExchange (15) : : : +- ^ FilterExecTransformer (13) - : : : +- ^ Scan parquet (12) + : : : +- ^ ScanTransformer parquet (12) : : +- ^ InputIteratorTransformer (27) : : +- BroadcastQueryStage (25), Statistics(X) : : +- ColumnarBroadcastExchange (24) : : +- ^ FilterExecTransformer (22) - : : +- ^ Scan parquet (21) + : : +- ^ ScanTransformer parquet (21) : +- ^ InputIteratorTransformer (36) : +- BroadcastQueryStage (34), Statistics(X) : +- ColumnarBroadcastExchange (33) : +- ^ FilterExecTransformer (31) - : +- ^ Scan parquet (30) + : +- ^ ScanTransformer parquet (30) +- ^ InputIteratorTransformer (42) +- BroadcastQueryStage (40), Statistics(X) +- ReusedExchange (39) @@ -84,7 +84,7 @@ AdaptiveSparkPlan (95) +- Scan parquet (85) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -113,7 +113,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (7) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(8) Scan parquet +(8) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,7 +134,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Input [7]: [s_suppkey#X, s_nationkey#X, l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(12) Scan parquet +(12) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +173,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] Input [7]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_orderkey#X, o_custkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -212,7 +212,7 @@ Join condition: None Output [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] Input [7]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X, c_custkey#X, c_nationkey#X] -(30) Scan parquet +(30) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt index d381d21da73d..9d1bc4a17558 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/8.txt @@ -34,40 +34,40 @@ AdaptiveSparkPlan (131) : : : : : : : +- ColumnarBroadcastExchange (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ FilterExecTransformer (10) - : : : : : : +- ^ Scan parquet (9) + : : : : : : +- ^ ScanTransformer parquet (9) : : : : : +- ^ InputIteratorTransformer (19) : : : : : +- BroadcastQueryStage (17), Statistics(X) : : : : : +- ColumnarBroadcastExchange (16) : : : : : +- ^ FilterExecTransformer (14) - : : : : : +- ^ Scan parquet (13) + : : : : : +- ^ ScanTransformer parquet (13) : : : : +- ^ InputIteratorTransformer (28) : : : : +- BroadcastQueryStage (26), Statistics(X) : : : : +- ColumnarBroadcastExchange (25) : : : : +- ^ FilterExecTransformer (23) - : : : : +- ^ Scan parquet (22) + : : : : +- ^ ScanTransformer parquet (22) : : : +- ^ InputIteratorTransformer (37) : : : +- BroadcastQueryStage (35), Statistics(X) : : : +- ColumnarBroadcastExchange (34) : : : +- ^ FilterExecTransformer (32) - : : : +- ^ Scan parquet (31) + : : : +- ^ ScanTransformer parquet (31) : : +- ^ InputIteratorTransformer (46) : : +- BroadcastQueryStage (44), Statistics(X) : : +- ColumnarBroadcastExchange (43) : : +- ^ FilterExecTransformer (41) - : : +- ^ Scan parquet (40) + : : +- ^ ScanTransformer parquet (40) : +- ^ InputIteratorTransformer (55) : +- BroadcastQueryStage (53), Statistics(X) : +- ColumnarBroadcastExchange (52) : +- ^ FilterExecTransformer (50) - : +- ^ Scan parquet (49) + : +- ^ ScanTransformer parquet (49) +- ^ InputIteratorTransformer (65) +- BroadcastQueryStage (63), Statistics(X) +- ColumnarBroadcastExchange (62) +- ^ ProjectExecTransformer (60) +- ^ FilterExecTransformer (59) - +- ^ Scan parquet (58) + +- ^ ScanTransformer parquet (58) +- == Initial Plan == Sort (130) +- Exchange (129) @@ -115,7 +115,7 @@ AdaptiveSparkPlan (131) +- Scan parquet (120) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -148,7 +148,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -169,7 +169,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Input [6]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -208,7 +208,7 @@ Join condition: None Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [6]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -247,7 +247,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] Input [7]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -286,7 +286,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X, c_custkey#X, c_nationkey#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -325,7 +325,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X, n_nationkey#X, n_regionkey#X] -(49) Scan parquet +(49) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -364,7 +364,7 @@ Join condition: None Output [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] Input [7]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X, n_nationkey#X, n_name#X] -(58) Scan parquet +(58) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt index f0ebba6efac5..8baec4075ad3 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-bhj/spark34/9.txt @@ -29,29 +29,29 @@ AdaptiveSparkPlan (100) : : : : : +- ColumnarBroadcastExchange (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ FilterExecTransformer (10) - : : : : +- ^ Scan parquet (9) + : : : : +- ^ ScanTransformer parquet (9) : : : +- ^ InputIteratorTransformer (19) : : : +- BroadcastQueryStage (17), Statistics(X) : : : +- ColumnarBroadcastExchange (16) : : : +- ^ FilterExecTransformer (14) - : : : +- ^ Scan parquet (13) + : : : +- ^ ScanTransformer parquet (13) : : +- ^ InputIteratorTransformer (28) : : +- BroadcastQueryStage (26), Statistics(X) : : +- ColumnarBroadcastExchange (25) : : +- ^ FilterExecTransformer (23) - : : +- ^ Scan parquet (22) + : : +- ^ ScanTransformer parquet (22) : +- ^ InputIteratorTransformer (37) : +- BroadcastQueryStage (35), Statistics(X) : +- ColumnarBroadcastExchange (34) : +- ^ FilterExecTransformer (32) - : +- ^ Scan parquet (31) + : +- ^ ScanTransformer parquet (31) +- ^ InputIteratorTransformer (46) +- BroadcastQueryStage (44), Statistics(X) +- ColumnarBroadcastExchange (43) +- ^ FilterExecTransformer (41) - +- ^ Scan parquet (40) + +- ^ ScanTransformer parquet (40) +- == Initial Plan == Sort (99) +- Exchange (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -121,7 +121,7 @@ Input [1]: [p_partkey#X] (8) InputIteratorTransformer Input [1]: [p_partkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +142,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [7]: [p_partkey#X, l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(13) Scan parquet +(13) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -181,7 +181,7 @@ Join condition: None Output [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] Input [8]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_suppkey#X, s_nationkey#X] -(22) Scan parquet +(22) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -220,7 +220,7 @@ Join condition: None Output [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] Input [10]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_partkey#X, ps_suppkey#X, ps_supplycost#X] -(31) Scan parquet +(31) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,7 +259,7 @@ Join condition: None Output [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] Input [8]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderkey#X, o_orderdate#X] -(40) Scan parquet +(40) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt index a7ced66a82e7..7c8451dfd6dd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt index b385b7719de2..f943422808e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt index 71dee80fab86..48e7e4d4e0d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt index edbcb50206df..c8ddb5437009 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt index 91df642595a3..40a891738143 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt index 56169a244bf9..9317c2f895a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/15.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (50) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -26,7 +26,7 @@ AdaptiveSparkPlan (50) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (49) +- Exchange (48) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (50) +- Scan parquet (38) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt index 8ef7d4d51441..2c41822beeeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt index fe915989f9e6..30fd4b0b1124 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +141,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt index a2345ca9ed35..dd4cad538e2c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt index d06cdb0fecff..507b700e37e9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt index 6fa1271a5a39..13c10a7bba29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/20.txt @@ -1,33 +1,33 @@ == Physical Plan == -AdaptiveSparkPlan (146) +AdaptiveSparkPlan (145) +- == Final Plan == - VeloxColumnarToRow (96) - +- ^ SortExecTransformer (94) - +- ^ InputIteratorTransformer (93) - +- ShuffleQueryStage (91) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (95) + +- ^ SortExecTransformer (93) + +- ^ InputIteratorTransformer (92) + +- ShuffleQueryStage (90) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) @@ -40,94 +40,93 @@ AdaptiveSparkPlan (146) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (145) - +- Exchange (144) - +- Project (143) - +- SortMergeJoin Inner (142) - :- Sort (136) - : +- Exchange (135) - : +- Project (134) - : +- SortMergeJoin LeftSemi (133) - : :- Sort (100) - : : +- Exchange (99) - : : +- Filter (98) - : : +- Scan parquet (97) - : +- Sort (132) - : +- Exchange (131) - : +- Project (130) - : +- SortMergeJoin Inner (129) - : :- Sort (112) - : : +- Exchange (111) - : : +- SortMergeJoin LeftSemi (110) - : : :- Sort (104) - : : : +- Exchange (103) - : : : +- Filter (102) - : : : +- Scan parquet (101) - : : +- Sort (109) - : : +- Exchange (108) - : : +- Project (107) - : : +- Filter (106) - : : +- Scan parquet (105) - : +- Sort (128) - : +- Exchange (127) - : +- Filter (126) - : +- HashAggregate (125) - : +- HashAggregate (124) - : +- SortMergeJoin LeftSemi (123) - : :- Sort (117) - : : +- Exchange (116) - : : +- Project (115) - : : +- Filter (114) - : : +- Scan parquet (113) - : +- Sort (122) - : +- Exchange (121) - : +- Project (120) - : +- Filter (119) - : +- Scan parquet (118) - +- Sort (141) - +- Exchange (140) - +- Project (139) - +- Filter (138) - +- Scan parquet (137) - - -(1) Scan parquet + Sort (144) + +- Exchange (143) + +- Project (142) + +- SortMergeJoin Inner (141) + :- Sort (135) + : +- Exchange (134) + : +- Project (133) + : +- SortMergeJoin LeftSemi (132) + : :- Sort (99) + : : +- Exchange (98) + : : +- Filter (97) + : : +- Scan parquet (96) + : +- Sort (131) + : +- Exchange (130) + : +- Project (129) + : +- SortMergeJoin Inner (128) + : :- Sort (111) + : : +- Exchange (110) + : : +- SortMergeJoin LeftSemi (109) + : : :- Sort (103) + : : : +- Exchange (102) + : : : +- Filter (101) + : : : +- Scan parquet (100) + : : +- Sort (108) + : : +- Exchange (107) + : : +- Project (106) + : : +- Filter (105) + : : +- Scan parquet (104) + : +- Sort (127) + : +- Exchange (126) + : +- Filter (125) + : +- HashAggregate (124) + : +- HashAggregate (123) + : +- SortMergeJoin LeftSemi (122) + : :- Sort (116) + : : +- Exchange (115) + : : +- Project (114) + : : +- Filter (113) + : : +- Scan parquet (112) + : +- Sort (121) + : +- Exchange (120) + : +- Project (119) + : +- Filter (118) + : +- Scan parquet (117) + +- Sort (140) + +- Exchange (139) + +- Project (138) + +- Filter (137) + +- Scan parquet (136) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -164,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -201,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -327,417 +326,410 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) InputAdapter +(91) InputAdapter Input [2]: [s_name#X, s_address#X] -(93) InputIteratorTransformer +(92) InputIteratorTransformer Input [2]: [s_name#X, s_address#X] -(94) SortExecTransformer +(93) SortExecTransformer Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(95) WholeStageCodegenTransformer (X) +(94) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(96) VeloxColumnarToRow +(95) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(97) Scan parquet +(96) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(98) Filter +(97) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(99) Exchange +(98) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) Sort +(99) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(101) Scan parquet +(100) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(102) Filter +(101) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(103) Exchange +(102) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(104) Sort +(103) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(105) Scan parquet +(104) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(106) Filter +(105) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(107) Project +(106) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(108) Exchange +(107) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(110) SortMergeJoin +(109) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(111) Exchange +(110) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(112) Sort +(111) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(113) Scan parquet +(112) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(114) Filter +(113) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(115) Project +(114) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(116) Exchange +(115) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(117) Sort +(116) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(118) Scan parquet +(117) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(119) Filter +(118) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(120) Project +(119) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(121) Exchange +(120) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(124) HashAggregate +(123) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(125) HashAggregate +(124) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(126) Filter +(125) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(127) Exchange +(126) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(128) Sort +(127) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(129) SortMergeJoin +(128) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(130) Project +(129) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(131) Exchange +(130) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(132) Sort +(131) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(133) SortMergeJoin +(132) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(134) Project +(133) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(135) Exchange +(134) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(136) Sort +(135) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(137) Scan parquet +(136) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(138) Filter +(137) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(139) Project +(138) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(140) Exchange +(139) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(141) Sort +(140) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(142) SortMergeJoin +(141) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(143) Project +(142) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(144) Exchange +(143) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(145) Sort +(144) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(146) AdaptiveSparkPlan +(145) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt index e70a145e9c1d..e72762dd9bfe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/21.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (138) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) @@ -44,34 +44,34 @@ AdaptiveSparkPlan (138) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (137) +- HashAggregate (136) @@ -120,7 +120,7 @@ AdaptiveSparkPlan (138) +- Scan parquet (127) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt index 674e0b96983f..81af91c0ee2a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt index 3b9b7fd93b96..2e427b6bf3ca 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt index ffb35d1813ae..40cfa020ce73 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt index 0e48f73c7795..37faa880d8af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt index f089996aa4e3..f3ec7ef3a620 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt index 80b0d02a6cb2..8bd2945e4859 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt index a41822a60a19..6cca8ba5a479 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark32/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt index 1b89e64860f3..a14969cbc44b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt index 6ca992b70988..55d544b898c6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -424,58 +424,57 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) - - -(83) Scan parquet + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) + + +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt index 346a6866342e..f998361cddc5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt index 9e927782f6c3..7c3509c3e726 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt index d82e4f2b21db..1161b32937bc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt index 950441af70e9..398cc1f5b8b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,34 +257,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt index aa7aa005a9b2..cac61db4a6bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt index e48379edc9fa..843ce9b39eb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +141,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt index 8dbf9429961a..1119ddcf80be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt index 41da1d28e987..31341db339e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt index 1fad7160bf64..9a2c56040b38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/20.txt @@ -1,32 +1,32 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -39,94 +39,93 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) - - -(1) Scan parquet + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -268,7 +267,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -326,407 +325,400 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt index fd61236faaf8..a675841a475a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt index 44c57da7482f..c48c13779d64 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -293,7 +293,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -400,7 +400,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt index a698975568fb..d09fe3e19a9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt index 8b00d867cd70..23fb123956dc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt index dd869d25d416..ea8f7dbd3371 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt index ac6dda065bb0..ccf77e181d75 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt index 6f5ed0d82b80..76e38b48de31 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt index 22cd2b9967fb..3c152f565673 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark33/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt index a052989ba525..1a64123cc8c0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt index 68e87c39a078..e2d5e58eba47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -428,58 +428,57 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) - - -(83) Scan parquet + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) + + +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join type: Inner Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt index 9175845083ca..3bfd39bdaf49 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt index b527c2bc3bc3..59abfb682c42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: LeftOuter Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt index cc6d364dea22..531cc5aaab55 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt index 8f1a1b4415db..f370146f9206 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,34 +259,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt index ee7826c8aa7e..3441216f140f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt index eaad0429ab40..c1b43b7fc421 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join type: Inner Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt index 85b11fa8ca18..c1a6b181bb34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -291,7 +290,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt index c72ca859ce37..4db9ca0c8393 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt index ed09215178a4..4d8eee1f1fb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/20.txt @@ -1,32 +1,32 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -39,94 +39,93 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) - - -(1) Scan parquet + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -328,415 +327,408 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt index 271d0c6d5098..96750b2b3ecc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt index b8123fb6dc0e..c6e147da04d8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -285,7 +285,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -295,7 +295,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -402,7 +402,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt index 8eea7581a417..1f86369a62af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -167,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt index 286881fb99b1..71a8f6974ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt index 0a9af460526c..da9339abd628 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt index aa69dbcd11f7..a0894489f0a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt index 05f9c2c90e5d..8ce3620f30d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt index e298d2a75196..4f537e91e132 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1-ras/spark34/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt index efe7f3908506..617489f1aa35 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt index a7ced66a82e7..7c8451dfd6dd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt index b385b7719de2..f943422808e8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt index 71dee80fab86..48e7e4d4e0d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt index edbcb50206df..c8ddb5437009 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt index 91df642595a3..40a891738143 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true) ELSE 0.0000 END)#X)), DecimalType(38,6), true)) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X as decimal(38,6)))), DecimalType(38,6), true) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt index 56169a244bf9..9317c2f895a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/15.txt @@ -15,7 +15,7 @@ AdaptiveSparkPlan (50) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -26,7 +26,7 @@ AdaptiveSparkPlan (50) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (49) +- Exchange (48) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (50) +- Scan parquet (38) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt index 8ef7d4d51441..2c41822beeeb 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt index fe915989f9e6..30fd4b0b1124 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +141,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7), true) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6), true) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt index a2345ca9ed35..dd4cad538e2c 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt index d06cdb0fecff..507b700e37e9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt index 6fa1271a5a39..13c10a7bba29 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/20.txt @@ -1,33 +1,33 @@ == Physical Plan == -AdaptiveSparkPlan (146) +AdaptiveSparkPlan (145) +- == Final Plan == - VeloxColumnarToRow (96) - +- ^ SortExecTransformer (94) - +- ^ InputIteratorTransformer (93) - +- ShuffleQueryStage (91) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (95) + +- ^ SortExecTransformer (93) + +- ^ InputIteratorTransformer (92) + +- ShuffleQueryStage (90) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33) : : +- ColumnarExchange (32) @@ -40,94 +40,93 @@ AdaptiveSparkPlan (146) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (145) - +- Exchange (144) - +- Project (143) - +- SortMergeJoin Inner (142) - :- Sort (136) - : +- Exchange (135) - : +- Project (134) - : +- SortMergeJoin LeftSemi (133) - : :- Sort (100) - : : +- Exchange (99) - : : +- Filter (98) - : : +- Scan parquet (97) - : +- Sort (132) - : +- Exchange (131) - : +- Project (130) - : +- SortMergeJoin Inner (129) - : :- Sort (112) - : : +- Exchange (111) - : : +- SortMergeJoin LeftSemi (110) - : : :- Sort (104) - : : : +- Exchange (103) - : : : +- Filter (102) - : : : +- Scan parquet (101) - : : +- Sort (109) - : : +- Exchange (108) - : : +- Project (107) - : : +- Filter (106) - : : +- Scan parquet (105) - : +- Sort (128) - : +- Exchange (127) - : +- Filter (126) - : +- HashAggregate (125) - : +- HashAggregate (124) - : +- SortMergeJoin LeftSemi (123) - : :- Sort (117) - : : +- Exchange (116) - : : +- Project (115) - : : +- Filter (114) - : : +- Scan parquet (113) - : +- Sort (122) - : +- Exchange (121) - : +- Project (120) - : +- Filter (119) - : +- Scan parquet (118) - +- Sort (141) - +- Exchange (140) - +- Project (139) - +- Filter (138) - +- Scan parquet (137) - - -(1) Scan parquet + Sort (144) + +- Exchange (143) + +- Project (142) + +- SortMergeJoin Inner (141) + :- Sort (135) + : +- Exchange (134) + : +- Project (133) + : +- SortMergeJoin LeftSemi (132) + : :- Sort (99) + : : +- Exchange (98) + : : +- Filter (97) + : : +- Scan parquet (96) + : +- Sort (131) + : +- Exchange (130) + : +- Project (129) + : +- SortMergeJoin Inner (128) + : :- Sort (111) + : : +- Exchange (110) + : : +- SortMergeJoin LeftSemi (109) + : : :- Sort (103) + : : : +- Exchange (102) + : : : +- Filter (101) + : : : +- Scan parquet (100) + : : +- Sort (108) + : : +- Exchange (107) + : : +- Project (106) + : : +- Filter (105) + : : +- Scan parquet (104) + : +- Sort (127) + : +- Exchange (126) + : +- Filter (125) + : +- HashAggregate (124) + : +- HashAggregate (123) + : +- SortMergeJoin LeftSemi (122) + : :- Sort (116) + : : +- Exchange (115) + : : +- Project (114) + : : +- Filter (113) + : : +- Scan parquet (112) + : +- Sort (121) + : +- Exchange (120) + : +- Project (119) + : +- Filter (118) + : +- Scan parquet (117) + +- Sort (140) + +- Exchange (139) + +- Project (138) + +- Filter (137) + +- Scan parquet (136) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -164,7 +163,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -201,7 +200,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -327,417 +326,410 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) InputAdapter +(91) InputAdapter Input [2]: [s_name#X, s_address#X] -(93) InputIteratorTransformer +(92) InputIteratorTransformer Input [2]: [s_name#X, s_address#X] -(94) SortExecTransformer +(93) SortExecTransformer Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(95) WholeStageCodegenTransformer (X) +(94) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(96) VeloxColumnarToRow +(95) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(97) Scan parquet +(96) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(98) Filter +(97) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(99) Exchange +(98) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) Sort +(99) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(101) Scan parquet +(100) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(102) Filter +(101) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(103) Exchange +(102) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(104) Sort +(103) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(105) Scan parquet +(104) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(106) Filter +(105) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(107) Project +(106) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(108) Exchange +(107) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(110) SortMergeJoin +(109) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(111) Exchange +(110) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(112) Sort +(111) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(113) Scan parquet +(112) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(114) Filter +(113) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(115) Project +(114) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(116) Exchange +(115) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(117) Sort +(116) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(118) Scan parquet +(117) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(119) Filter +(118) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(120) Project +(119) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(121) Exchange +(120) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(124) HashAggregate +(123) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(125) HashAggregate +(124) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3), true) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(126) Filter +(125) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(127) Exchange +(126) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(128) Sort +(127) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(129) SortMergeJoin +(128) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(cast(ps_availqty#X as decimal(10,0)) as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(130) Project +(129) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(131) Exchange +(130) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(132) Sort +(131) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(133) SortMergeJoin +(132) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(134) Project +(133) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(135) Exchange +(134) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(136) Sort +(135) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(137) Scan parquet +(136) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(138) Filter +(137) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(139) Project +(138) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(140) Exchange +(139) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(141) Sort +(140) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(142) SortMergeJoin +(141) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(143) Project +(142) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(144) Exchange +(143) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(145) Sort +(144) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(146) AdaptiveSparkPlan +(145) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt index e70a145e9c1d..e72762dd9bfe 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/21.txt @@ -30,7 +30,7 @@ AdaptiveSparkPlan (138) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42) : : +- ColumnarExchange (41) @@ -44,34 +44,34 @@ AdaptiveSparkPlan (138) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (137) +- HashAggregate (136) @@ -120,7 +120,7 @@ AdaptiveSparkPlan (138) +- Scan parquet (127) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -157,7 +157,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -330,7 +330,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -398,7 +398,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt index 674e0b96983f..81af91c0ee2a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt index 3b9b7fd93b96..2e427b6bf3ca 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2), true))), DecimalType(26,4), true))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt index ffb35d1813ae..40cfa020ce73 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt index 0e48f73c7795..37faa880d8af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt index a1830678d0de..73d9d734e78a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt index f089996aa4e3..f3ec7ef3a620 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt index 80b0d02a6cb2..8bd2945e4859 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt index a41822a60a19..6cca8ba5a479 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark32/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt index 0851fa2447ea..ca3b50ab05c5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt index 1b89e64860f3..a14969cbc44b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -261,7 +261,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt index 6ca992b70988..55d544b898c6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -177,7 +177,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -424,58 +424,57 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) - - -(83) Scan parquet + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) + + +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -581,129 +580,122 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X] Results [1]: [CheckOverflow((promote_precision(cast(sum(CheckOverflow((promote_precision(ps_supplycost#X) * promote_precision(cast(ps_availqty#X as decimal(12,2)))), DecimalType(23,2)))#X as decimal(38,10))) * 0.0001000000), DecimalType(38,6)) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt index 346a6866342e..f998361cddc5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt index 9e927782f6c3..7c3509c3e726 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -133,180 +132,173 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt index d82e4f2b21db..1161b32937bc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,93 +122,86 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END), sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [CheckOverflow((promote_precision(CheckOverflow((100.0000 * promote_precision(sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)) ELSE 0.0000 END)#X)), DecimalType(38,6))) / promote_precision(cast(sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X as decimal(38,6)))), DecimalType(38,6)) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt index 950441af70e9..398cc1f5b8b9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -257,34 +257,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -346,71 +345,64 @@ Input [2]: [l_suppkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedpri (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt index aa7aa005a9b2..cac61db4a6bd 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt index e48379edc9fa..843ce9b39eb6 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -142,7 +141,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -213,139 +212,132 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [CheckOverflow((0.200000 * promote_precision(avg(l_quantity#X)#X)), DecimalType(18,7)) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [CheckOverflow((promote_precision(sum(l_extendedprice#X)#X) / 7.00), DecimalType(27,6)) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt index 8dbf9429961a..1119ddcf80be 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -289,7 +288,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -371,219 +370,212 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt index 41da1d28e987..31341db339e1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -122,89 +121,82 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt index 1fad7160bf64..9a2c56040b38 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/20.txt @@ -1,32 +1,32 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -39,94 +39,93 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) - - -(1) Scan parquet + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -268,7 +267,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -326,407 +325,400 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [CheckOverflow((0.50 * promote_precision(sum(l_quantity#X)#X)), DecimalType(24,3)) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt index fd61236faaf8..a675841a475a 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -230,7 +230,7 @@ Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -329,7 +329,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -397,7 +397,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt index 44c57da7482f..c48c13779d64 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -293,7 +293,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -400,7 +400,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt index a698975568fb..d09fe3e19a9b 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -166,7 +165,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))] Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X] Results [4]: [l_orderkey#X, sum(CheckOverflow((promote_precision(cast(l_extendedprice#X as decimal(13,2))) * promote_precision(CheckOverflow((1.00 - promote_precision(cast(l_discount#X as decimal(13,2)))), DecimalType(13,2)))), DecimalType(26,4)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt index 8b00d867cd70..23fb123956dc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt index dd869d25d416..ea8f7dbd3371 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -307,7 +307,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,7 +375,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -443,7 +443,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt index 5df63675caee..d5569132a41f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt index ac6dda065bb0..ccf77e181d75 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -233,7 +233,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -301,7 +301,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -369,7 +369,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt index 6f5ed0d82b80..76e38b48de31 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -282,7 +282,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -350,7 +350,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -418,7 +418,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -486,7 +486,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -554,7 +554,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -622,7 +622,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt index 22cd2b9967fb..3c152f565673 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark33/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -238,7 +238,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -306,7 +306,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -374,7 +374,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -442,7 +442,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt index 6dbee1ad7ffb..4d61f904dbe9 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/1.txt @@ -16,7 +16,7 @@ AdaptiveSparkPlan (30) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == Sort (29) +- Exchange (28) @@ -28,7 +28,7 @@ AdaptiveSparkPlan (30) +- Scan parquet (22) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_tax#X, l_returnflag#X, l_linestatus#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt index a052989ba525..1a64123cc8c0 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/10.txt @@ -31,28 +31,28 @@ AdaptiveSparkPlan (100) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (18) : : +- ShuffleQueryStage (16), Statistics(X) : : +- ColumnarExchange (15) : : +- VeloxResizeBatches (14) : : +- ^ ProjectExecTransformer (12) : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) + : : +- ^ ScanTransformer parquet (10) : +- ^ InputIteratorTransformer (35) : +- ShuffleQueryStage (33), Statistics(X) : +- ColumnarExchange (32) : +- VeloxResizeBatches (31) : +- ^ ProjectExecTransformer (29) : +- ^ FilterExecTransformer (28) - : +- ^ Scan parquet (27) + : +- ^ ScanTransformer parquet (27) +- ^ InputIteratorTransformer (52) +- ShuffleQueryStage (50), Statistics(X) +- ColumnarExchange (49) +- VeloxResizeBatches (48) +- ^ ProjectExecTransformer (46) +- ^ FilterExecTransformer (45) - +- ^ Scan parquet (44) + +- ^ ScanTransformer parquet (44) +- == Initial Plan == TakeOrderedAndProject (99) +- HashAggregate (98) @@ -88,7 +88,7 @@ AdaptiveSparkPlan (100) +- Scan parquet (90) -(1) Scan parquet +(1) ScanTransformer parquet Output [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -125,7 +125,7 @@ Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (9) InputIteratorTransformer Input [7]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -194,7 +194,7 @@ Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (26) InputIteratorTransformer Input [8]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_returnflag#X] Batched: true Location: InMemoryFileIndex [*] @@ -263,7 +263,7 @@ Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acct (43) InputIteratorTransformer Input [9]: [c_custkey#X, c_name#X, c_address#X, c_nationkey#X, c_phone#X, c_acctbal#X, c_comment#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt index 68e87c39a078..e2d5e58eba47 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/11.txt @@ -29,21 +29,21 @@ AdaptiveSparkPlan (82) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) + : : +- ^ ScanTransformer parquet (1) : +- ^ InputIteratorTransformer (18) : +- ShuffleQueryStage (16), Statistics(X) : +- ColumnarExchange (15) : +- VeloxResizeBatches (14) : +- ^ ProjectExecTransformer (12) : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) + : +- ^ ScanTransformer parquet (10) +- ^ InputIteratorTransformer (35) +- ShuffleQueryStage (33), Statistics(X) +- ColumnarExchange (32) +- VeloxResizeBatches (31) +- ^ ProjectExecTransformer (29) +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == Sort (81) +- Exchange (80) @@ -72,7 +72,7 @@ AdaptiveSparkPlan (82) +- Scan parquet (69) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -109,7 +109,7 @@ Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] (9) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, ps_supplycost#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -178,7 +178,7 @@ Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] (26) InputIteratorTransformer Input [4]: [ps_partkey#X, ps_availqty#X, ps_supplycost#X, s_nationkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -428,58 +428,57 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (136) +AdaptiveSparkPlan (135) +- == Final Plan == - VeloxColumnarToRow (114) - +- ^ ProjectExecTransformer (112) - +- ^ RegularHashAggregateExecTransformer (111) - +- ^ RegularHashAggregateExecTransformer (110) - +- ^ ProjectExecTransformer (109) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) - :- ^ InputIteratorTransformer (103) - : +- ShuffleQueryStage (101), Statistics(X) - : +- ColumnarExchange (100) - : +- VeloxResizeBatches (99) - : +- ^ ProjectExecTransformer (97) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) - : :- ^ InputIteratorTransformer (91) - : : +- ShuffleQueryStage (89), Statistics(X) - : : +- ColumnarExchange (88) - : : +- VeloxResizeBatches (87) - : : +- ^ ProjectExecTransformer (85) - : : +- ^ FilterExecTransformer (84) - : : +- ^ Scan parquet (83) - : +- ^ InputIteratorTransformer (95) - : +- ShuffleQueryStage (93), Statistics(X) - : +- ReusedExchange (92) - +- ^ InputIteratorTransformer (107) - +- ShuffleQueryStage (105), Statistics(X) - +- ReusedExchange (104) + VeloxColumnarToRow (113) + +- ^ ProjectExecTransformer (111) + +- ^ RegularHashAggregateExecTransformer (110) + +- ^ ProjectExecTransformer (109) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (108) + :- ^ InputIteratorTransformer (103) + : +- ShuffleQueryStage (101), Statistics(X) + : +- ColumnarExchange (100) + : +- VeloxResizeBatches (99) + : +- ^ ProjectExecTransformer (97) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (96) + : :- ^ InputIteratorTransformer (91) + : : +- ShuffleQueryStage (89), Statistics(X) + : : +- ColumnarExchange (88) + : : +- VeloxResizeBatches (87) + : : +- ^ ProjectExecTransformer (85) + : : +- ^ FilterExecTransformer (84) + : : +- ^ ScanTransformer parquet (83) + : +- ^ InputIteratorTransformer (95) + : +- ShuffleQueryStage (93), Statistics(X) + : +- ReusedExchange (92) + +- ^ InputIteratorTransformer (107) + +- ShuffleQueryStage (105), Statistics(X) + +- ReusedExchange (104) +- == Initial Plan == - HashAggregate (135) - +- HashAggregate (134) - +- Project (133) - +- SortMergeJoin Inner (132) - :- Sort (126) - : +- Exchange (125) - : +- Project (124) - : +- SortMergeJoin Inner (123) - : :- Sort (118) - : : +- Exchange (117) - : : +- Filter (116) - : : +- Scan parquet (115) - : +- Sort (122) - : +- Exchange (121) - : +- Filter (120) - : +- Scan parquet (119) - +- Sort (131) - +- Exchange (130) - +- Project (129) - +- Filter (128) - +- Scan parquet (127) - - -(83) Scan parquet + HashAggregate (134) + +- HashAggregate (133) + +- Project (132) + +- SortMergeJoin Inner (131) + :- Sort (125) + : +- Exchange (124) + : +- Project (123) + : +- SortMergeJoin Inner (122) + : :- Sort (117) + : : +- Exchange (116) + : : +- Filter (115) + : : +- Scan parquet (114) + : +- Sort (121) + : +- Exchange (120) + : +- Filter (119) + : +- Scan parquet (118) + +- Sort (130) + +- Exchange (129) + +- Project (128) + +- Filter (127) + +- Scan parquet (126) + + +(83) ScanTransformer parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -587,131 +586,124 @@ Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] (110) RegularHashAggregateExecTransformer Input [3]: [ps_availqty#X, ps_supplycost#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(111) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(112) ProjectExecTransformer +(111) ProjectExecTransformer Output [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Input [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] -(113) WholeStageCodegenTransformer (X) +(112) WholeStageCodegenTransformer (X) Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: false -(114) VeloxColumnarToRow +(113) VeloxColumnarToRow Input [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(115) Scan parquet +(114) Scan parquet Output [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_suppkey)] ReadSchema: struct -(116) Filter +(115) Filter Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Condition : isnotnull(ps_suppkey#X) -(117) Exchange +(116) Exchange Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(118) Sort +(117) Sort Input [3]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(119) Scan parquet +(118) Scan parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_suppkey), IsNotNull(s_nationkey)] ReadSchema: struct -(120) Filter +(119) Filter Input [2]: [s_suppkey#X, s_nationkey#X] Condition : (isnotnull(s_suppkey#X) AND isnotnull(s_nationkey#X)) -(121) Exchange +(120) Exchange Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(122) Sort +(121) Sort Input [2]: [s_suppkey#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(123) SortMergeJoin +(122) SortMergeJoin Left keys [1]: [ps_suppkey#X] Right keys [1]: [s_suppkey#X] Join type: Inner Join condition: None -(124) Project +(123) Project Output [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Input [5]: [ps_suppkey#X, ps_availqty#X, ps_supplycost#X, s_suppkey#X, s_nationkey#X] -(125) Exchange +(124) Exchange Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(126) Sort +(125) Sort Input [3]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(127) Scan parquet +(126) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,GERMANY), IsNotNull(n_nationkey)] ReadSchema: struct -(128) Filter +(127) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = GERMANY)) AND isnotnull(n_nationkey#X)) -(129) Project +(128) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(130) Exchange +(129) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(131) Sort +(130) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(132) SortMergeJoin +(131) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(133) Project +(132) Project Output [2]: [ps_availqty#X, ps_supplycost#X] Input [4]: [ps_availqty#X, ps_supplycost#X, s_nationkey#X, n_nationkey#X] -(134) HashAggregate +(133) HashAggregate Input [2]: [ps_availqty#X, ps_supplycost#X] Keys: [] Functions [1]: [partial_sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(135) HashAggregate +(134) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))] Aggregate Attributes [1]: [sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X] Results [1]: [(sum((ps_supplycost#X * cast(ps_availqty#X as decimal(10,0))))#X * 0.0001000000) AS (sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] -(136) AdaptiveSparkPlan +(135) AdaptiveSparkPlan Output [1]: [(sum((ps_supplycost * ps_availqty)) * 0.0001000000)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt index 9175845083ca..3bfd39bdaf49 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/12.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (55) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (54) +- Exchange (53) @@ -49,7 +49,7 @@ AdaptiveSparkPlan (55) +- Scan parquet (43) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -86,7 +86,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_shipdate#X, l_commitdate#X, l_receiptdate#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt index b527c2bc3bc3..59abfb682c42 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/13.txt @@ -1,58 +1,57 @@ == Physical Plan == -AdaptiveSparkPlan (58) +AdaptiveSparkPlan (57) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ SortExecTransformer (38) - +- ^ InputIteratorTransformer (37) - +- ShuffleQueryStage (35), Statistics(X) - +- ColumnarExchange (34) - +- VeloxResizeBatches (33) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ ProjectExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ RegularHashAggregateExecTransformer (20) - +- ^ ProjectExecTransformer (19) - +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) - :- ^ InputIteratorTransformer (8) - : +- ShuffleQueryStage (6), Statistics(X) - : +- ColumnarExchange (5) - : +- VeloxResizeBatches (4) - : +- ^ ProjectExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (17) - +- ShuffleQueryStage (15), Statistics(X) - +- ColumnarExchange (14) - +- VeloxResizeBatches (13) - +- ^ ProjectExecTransformer (11) - +- ^ FilterExecTransformer (10) - +- ^ Scan parquet (9) + VeloxColumnarToRow (39) + +- ^ SortExecTransformer (37) + +- ^ InputIteratorTransformer (36) + +- ShuffleQueryStage (34), Statistics(X) + +- ColumnarExchange (33) + +- VeloxResizeBatches (32) + +- ^ RegularHashAggregateExecTransformer (30) + +- ^ InputIteratorTransformer (29) + +- ShuffleQueryStage (27), Statistics(X) + +- ColumnarExchange (26) + +- VeloxResizeBatches (25) + +- ^ ProjectExecTransformer (23) + +- ^ FlushableHashAggregateExecTransformer (22) + +- ^ ProjectExecTransformer (21) + +- ^ RegularHashAggregateExecTransformer (20) + +- ^ ProjectExecTransformer (19) + +- ^ ShuffledHashJoinExecTransformer LeftOuter BuildLeft (18) + :- ^ InputIteratorTransformer (8) + : +- ShuffleQueryStage (6), Statistics(X) + : +- ColumnarExchange (5) + : +- VeloxResizeBatches (4) + : +- ^ ProjectExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (17) + +- ShuffleQueryStage (15), Statistics(X) + +- ColumnarExchange (14) + +- VeloxResizeBatches (13) + +- ^ ProjectExecTransformer (11) + +- ^ FilterExecTransformer (10) + +- ^ ScanTransformer parquet (9) +- == Initial Plan == - Sort (57) - +- Exchange (56) - +- HashAggregate (55) - +- Exchange (54) - +- HashAggregate (53) - +- HashAggregate (52) - +- HashAggregate (51) - +- Project (50) - +- SortMergeJoin LeftOuter (49) - :- Sort (43) - : +- Exchange (42) - : +- Scan parquet (41) - +- Sort (48) - +- Exchange (47) - +- Project (46) - +- Filter (45) - +- Scan parquet (44) - - -(1) Scan parquet + Sort (56) + +- Exchange (55) + +- HashAggregate (54) + +- Exchange (53) + +- HashAggregate (52) + +- HashAggregate (51) + +- HashAggregate (50) + +- Project (49) + +- SortMergeJoin LeftOuter (48) + :- Sort (42) + : +- Exchange (41) + : +- Scan parquet (40) + +- Sort (47) + +- Exchange (46) + +- Project (45) + +- Filter (44) + +- Scan parquet (43) + + +(1) ScanTransformer parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -84,7 +83,7 @@ Input [1]: [c_custkey#X] (8) InputIteratorTransformer Input [1]: [c_custkey#X] -(9) Scan parquet +(9) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] @@ -134,181 +133,174 @@ Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] (20) RegularHashAggregateExecTransformer Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] -Functions [1]: [partial_count(o_orderkey#X)] -Aggregate Attributes [1]: [count#X] -Results [2]: [c_custkey#X, count#X] - -(21) RegularHashAggregateExecTransformer -Input [2]: [c_custkey#X, count#X] -Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [2]: [c_custkey#X, count(o_orderkey#X)#X] -(22) ProjectExecTransformer +(21) ProjectExecTransformer Output [1]: [count(o_orderkey#X)#X AS c_count#X] Input [2]: [c_custkey#X, count(o_orderkey#X)#X] -(23) FlushableHashAggregateExecTransformer +(22) FlushableHashAggregateExecTransformer Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(24) ProjectExecTransformer +(23) ProjectExecTransformer Output [3]: [hash(c_count#X, 42) AS hash_partition_key#X, c_count#X, count#X] Input [2]: [c_count#X, count#X] -(25) WholeStageCodegenTransformer (X) +(24) WholeStageCodegenTransformer (X) Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: false -(26) VeloxResizeBatches +(25) VeloxResizeBatches Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: X, X -(27) ColumnarExchange +(26) ColumnarExchange Input [3]: [hash_partition_key#X, c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [c_count#X, count#X], [plan_id=X], [shuffle_writer_type=hash] -(28) ShuffleQueryStage +(27) ShuffleQueryStage Output [2]: [c_count#X, count#X] Arguments: X -(29) InputAdapter +(28) InputAdapter Input [2]: [c_count#X, count#X] -(30) InputIteratorTransformer +(29) InputIteratorTransformer Input [2]: [c_count#X, count#X] -(31) RegularHashAggregateExecTransformer +(30) RegularHashAggregateExecTransformer Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(32) WholeStageCodegenTransformer (X) +(31) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(33) VeloxResizeBatches +(32) VeloxResizeBatches Input [2]: [c_count#X, custdist#X] Arguments: X, X -(34) ColumnarExchange +(33) ColumnarExchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(35) ShuffleQueryStage +(34) ShuffleQueryStage Output [2]: [c_count#X, custdist#X] Arguments: X -(36) InputAdapter +(35) InputAdapter Input [2]: [c_count#X, custdist#X] -(37) InputIteratorTransformer +(36) InputIteratorTransformer Input [2]: [c_count#X, custdist#X] -(38) SortExecTransformer +(37) SortExecTransformer Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [2]: [c_count#X, custdist#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [2]: [c_count#X, custdist#X] -(41) Scan parquet +(40) Scan parquet Output [1]: [c_custkey#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(42) Exchange +(41) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(43) Sort +(42) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(44) Scan parquet +(43) Scan parquet Output [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_comment), IsNotNull(o_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] Condition : ((isnotnull(o_comment#X) AND NOT o_comment#X LIKE %special%requests%) AND isnotnull(o_custkey#X)) -(46) Project +(45) Project Output [2]: [o_orderkey#X, o_custkey#X] Input [3]: [o_orderkey#X, o_custkey#X, o_comment#X] -(47) Exchange +(46) Exchange Input [2]: [o_orderkey#X, o_custkey#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [2]: [o_orderkey#X, o_custkey#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(49) SortMergeJoin +(48) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: LeftOuter Join condition: None -(50) Project +(49) Project Output [2]: [c_custkey#X, o_orderkey#X] Input [3]: [c_custkey#X, o_orderkey#X, o_custkey#X] -(51) HashAggregate +(50) HashAggregate Input [2]: [c_custkey#X, o_orderkey#X] Keys [1]: [c_custkey#X] Functions [1]: [partial_count(o_orderkey#X)] Aggregate Attributes [1]: [count#X] Results [2]: [c_custkey#X, count#X] -(52) HashAggregate +(51) HashAggregate Input [2]: [c_custkey#X, count#X] Keys [1]: [c_custkey#X] Functions [1]: [count(o_orderkey#X)] Aggregate Attributes [1]: [count(o_orderkey#X)#X] Results [1]: [count(o_orderkey#X)#X AS c_count#X] -(53) HashAggregate +(52) HashAggregate Input [1]: [c_count#X] Keys [1]: [c_count#X] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#X] Results [2]: [c_count#X, count#X] -(54) Exchange +(53) Exchange Input [2]: [c_count#X, count#X] Arguments: hashpartitioning(c_count#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(55) HashAggregate +(54) HashAggregate Input [2]: [c_count#X, count#X] Keys [1]: [c_count#X] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#X] Results [2]: [c_count#X, count(1)#X AS custdist#X] -(56) Exchange +(55) Exchange Input [2]: [c_count#X, custdist#X] Arguments: rangepartitioning(custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(57) Sort +(56) Sort Input [2]: [c_count#X, custdist#X] Arguments: [custdist#X DESC NULLS LAST, c_count#X DESC NULLS LAST], true, 0 -(58) AdaptiveSparkPlan +(57) AdaptiveSparkPlan Output [2]: [c_count#X, custdist#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt index cc6d364dea22..531cc5aaab55 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/14.txt @@ -1,43 +1,42 @@ == Physical Plan == -AdaptiveSparkPlan (39) +AdaptiveSparkPlan (38) +- == Final Plan == - VeloxColumnarToRow (25) - +- ^ ProjectExecTransformer (23) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (24) + +- ^ ProjectExecTransformer (22) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (38) - +- HashAggregate (37) - +- Project (36) - +- SortMergeJoin Inner (35) - :- Sort (30) - : +- Exchange (29) - : +- Project (28) - : +- Filter (27) - : +- Scan parquet (26) - +- Sort (34) - +- Exchange (33) - +- Filter (32) - +- Scan parquet (31) - - -(1) Scan parquet + HashAggregate (37) + +- HashAggregate (36) + +- Project (35) + +- SortMergeJoin Inner (34) + :- Sort (29) + : +- Exchange (28) + : +- Project (27) + : +- Filter (26) + : +- Scan parquet (25) + +- Sort (33) + +- Exchange (32) + +- Filter (31) + +- Scan parquet (30) + + +(1) ScanTransformer parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -74,7 +73,7 @@ Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -124,94 +123,87 @@ Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] (21) RegularHashAggregateExecTransformer Input [5]: [l_extendedprice#X, l_discount#X, p_type#X, _pre_X#X, _pre_X#X] Keys: [] -Functions [2]: [partial_sum(_pre_X#X), partial_sum(_pre_X#X)] -Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -Keys: [] -Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [2]: [sum(_pre_X#X), sum(_pre_X#X)] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(23) ProjectExecTransformer +(22) ProjectExecTransformer Output [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] Input [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(24) WholeStageCodegenTransformer (X) +(23) WholeStageCodegenTransformer (X) Input [1]: [promo_revenue#X] Arguments: false -(25) VeloxColumnarToRow +(24) VeloxColumnarToRow Input [1]: [promo_revenue#X] -(26) Scan parquet +(25) Scan parquet Output [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1995-09-01), LessThan(l_shipdate,1995-10-01), IsNotNull(l_partkey)] ReadSchema: struct -(27) Filter +(26) Filter Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : (((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1995-09-01)) AND (l_shipdate#X < 1995-10-01)) AND isnotnull(l_partkey#X)) -(28) Project +(27) Project Output [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_partkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(29) Exchange +(28) Exchange Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(30) Sort +(29) Sort Input [3]: [l_partkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(31) Scan parquet +(30) Scan parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_partkey)] ReadSchema: struct -(32) Filter +(31) Filter Input [2]: [p_partkey#X, p_type#X] Condition : isnotnull(p_partkey#X) -(33) Exchange +(32) Exchange Input [2]: [p_partkey#X, p_type#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(34) Sort +(33) Sort Input [2]: [p_partkey#X, p_type#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(35) SortMergeJoin +(34) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(36) Project +(35) Project Output [3]: [l_extendedprice#X, l_discount#X, p_type#X] Input [5]: [l_partkey#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_type#X] -(37) HashAggregate +(36) HashAggregate Input [3]: [l_extendedprice#X, l_discount#X, p_type#X] Keys: [] Functions [2]: [partial_sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Results [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] -(38) HashAggregate +(37) HashAggregate Input [4]: [sum#X, isEmpty#X, sum#X, isEmpty#X] Keys: [] Functions [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END), sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [((100.00 * sum(CASE WHEN StartsWith(p_type#X, PROMO) THEN (l_extendedprice#X * (1 - l_discount#X)) ELSE 0.0000 END)#X) / sum((l_extendedprice#X * (1 - l_discount#X)))#X) AS promo_revenue#X] -(39) AdaptiveSparkPlan +(38) AdaptiveSparkPlan Output [1]: [promo_revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt index 8f1a1b4415db..f370146f9206 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/15.txt @@ -14,7 +14,7 @@ AdaptiveSparkPlan (47) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ FilterExecTransformer (22) +- ^ RegularHashAggregateExecTransformer (21) +- ^ InputIteratorTransformer (20) @@ -25,7 +25,7 @@ AdaptiveSparkPlan (47) +- ^ FlushableHashAggregateExecTransformer (13) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (46) +- Exchange (45) @@ -45,7 +45,7 @@ AdaptiveSparkPlan (47) +- Scan parquet (35) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] Batched: true Location: InMemoryFileIndex [*] @@ -82,7 +82,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_phone#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -259,34 +259,33 @@ Arguments: isFinalPlan=true ===== Subqueries ===== Subquery:1 Hosting operator id = 22 Hosting Expression = Subquery subquery#X, [id=#X] -AdaptiveSparkPlan (73) +AdaptiveSparkPlan (72) +- == Final Plan == - VeloxColumnarToRow (64) - +- ^ RegularHashAggregateExecTransformer (62) - +- ^ RegularHashAggregateExecTransformer (61) - +- ^ ProjectExecTransformer (60) - +- ^ RegularHashAggregateExecTransformer (59) - +- ^ InputIteratorTransformer (58) - +- ShuffleQueryStage (56), Statistics(X) - +- ColumnarExchange (55) - +- VeloxResizeBatches (54) - +- ^ ProjectExecTransformer (52) - +- ^ FlushableHashAggregateExecTransformer (51) - +- ^ ProjectExecTransformer (50) - +- ^ FilterExecTransformer (49) - +- ^ Scan parquet (48) + VeloxColumnarToRow (63) + +- ^ RegularHashAggregateExecTransformer (61) + +- ^ ProjectExecTransformer (60) + +- ^ RegularHashAggregateExecTransformer (59) + +- ^ InputIteratorTransformer (58) + +- ShuffleQueryStage (56), Statistics(X) + +- ColumnarExchange (55) + +- VeloxResizeBatches (54) + +- ^ ProjectExecTransformer (52) + +- ^ FlushableHashAggregateExecTransformer (51) + +- ^ ProjectExecTransformer (50) + +- ^ FilterExecTransformer (49) + +- ^ ScanTransformer parquet (48) +- == Initial Plan == - HashAggregate (72) - +- HashAggregate (71) - +- HashAggregate (70) - +- Exchange (69) - +- HashAggregate (68) - +- Project (67) - +- Filter (66) - +- Scan parquet (65) + HashAggregate (71) + +- HashAggregate (70) + +- HashAggregate (69) + +- Exchange (68) + +- HashAggregate (67) + +- Project (66) + +- Filter (65) + +- Scan parquet (64) -(48) Scan parquet +(48) ScanTransformer parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -348,71 +347,64 @@ Input [2]: [l_suppkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] (61) RegularHashAggregateExecTransformer Input [1]: [total_revenue#X] Keys: [] -Functions [1]: [partial_max(total_revenue#X)] -Aggregate Attributes [1]: [max#X] -Results [1]: [max#X] - -(62) RegularHashAggregateExecTransformer -Input [1]: [max#X] -Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [1]: [max(total_revenue)#X] Arguments: false -(64) VeloxColumnarToRow +(63) VeloxColumnarToRow Input [1]: [max(total_revenue)#X] -(65) Scan parquet +(64) Scan parquet Output [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1996-01-01), LessThan(l_shipdate,1996-04-01)] ReadSchema: struct -(66) Filter +(65) Filter Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1996-01-01)) AND (l_shipdate#X < 1996-04-01)) -(67) Project +(66) Project Output [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(68) HashAggregate +(67) HashAggregate Input [3]: [l_suppkey#X, l_extendedprice#X, l_discount#X] Keys [1]: [l_suppkey#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_suppkey#X, sum#X, isEmpty#X] -(69) Exchange +(68) Exchange Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(70) HashAggregate +(69) HashAggregate Input [3]: [l_suppkey#X, sum#X, isEmpty#X] Keys [1]: [l_suppkey#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS total_revenue#X] -(71) HashAggregate +(70) HashAggregate Input [1]: [total_revenue#X] Keys: [] Functions [1]: [partial_max(total_revenue#X)] Aggregate Attributes [1]: [max#X] Results [1]: [max#X] -(72) HashAggregate +(71) HashAggregate Input [1]: [max#X] Keys: [] Functions [1]: [max(total_revenue#X)] Aggregate Attributes [1]: [max(total_revenue#X)#X] Results [1]: [max(total_revenue#X)#X AS max(total_revenue)#X] -(73) AdaptiveSparkPlan +(72) AdaptiveSparkPlan Output [1]: [max(total_revenue)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt index ee7826c8aa7e..3441216f140f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/16.txt @@ -29,14 +29,14 @@ AdaptiveSparkPlan (71) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (70) +- Exchange (69) @@ -63,7 +63,7 @@ AdaptiveSparkPlan (71) +- Scan parquet (57) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [ps_partkey#X, ps_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -100,7 +100,7 @@ Input [2]: [ps_partkey#X, ps_suppkey#X] (9) InputIteratorTransformer Input [2]: [ps_partkey#X, ps_suppkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_type#X, p_size#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt index eaad0429ab40..c1b43b7fc421 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/17.txt @@ -1,65 +1,64 @@ == Physical Plan == -AdaptiveSparkPlan (63) +AdaptiveSparkPlan (62) +- == Final Plan == - VeloxColumnarToRow (40) - +- ^ ProjectExecTransformer (38) - +- ^ RegularHashAggregateExecTransformer (37) - +- ^ RegularHashAggregateExecTransformer (36) - +- ^ ProjectExecTransformer (35) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) - :- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ FilterExecTransformer (33) - +- ^ ProjectExecTransformer (32) - +- ^ RegularHashAggregateExecTransformer (31) - +- ^ InputIteratorTransformer (30) - +- ShuffleQueryStage (28), Statistics(X) - +- ColumnarExchange (27) - +- VeloxResizeBatches (26) - +- ^ ProjectExecTransformer (24) - +- ^ FlushableHashAggregateExecTransformer (23) - +- ^ FilterExecTransformer (22) - +- ^ Scan parquet (21) + VeloxColumnarToRow (39) + +- ^ ProjectExecTransformer (37) + +- ^ RegularHashAggregateExecTransformer (36) + +- ^ ProjectExecTransformer (35) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (34) + :- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ FilterExecTransformer (33) + +- ^ ProjectExecTransformer (32) + +- ^ RegularHashAggregateExecTransformer (31) + +- ^ InputIteratorTransformer (30) + +- ShuffleQueryStage (28), Statistics(X) + +- ColumnarExchange (27) + +- VeloxResizeBatches (26) + +- ^ ProjectExecTransformer (24) + +- ^ FlushableHashAggregateExecTransformer (23) + +- ^ FilterExecTransformer (22) + +- ^ ScanTransformer parquet (21) +- == Initial Plan == - HashAggregate (62) - +- HashAggregate (61) - +- Project (60) - +- SortMergeJoin Inner (59) - :- Project (51) - : +- SortMergeJoin Inner (50) - : :- Sort (44) - : : +- Exchange (43) - : : +- Filter (42) - : : +- Scan parquet (41) - : +- Sort (49) - : +- Exchange (48) - : +- Project (47) - : +- Filter (46) - : +- Scan parquet (45) - +- Sort (58) - +- Filter (57) - +- HashAggregate (56) - +- Exchange (55) - +- HashAggregate (54) - +- Filter (53) - +- Scan parquet (52) - - -(1) Scan parquet + HashAggregate (61) + +- HashAggregate (60) + +- Project (59) + +- SortMergeJoin Inner (58) + :- Project (50) + : +- SortMergeJoin Inner (49) + : :- Sort (43) + : : +- Exchange (42) + : : +- Filter (41) + : : +- Scan parquet (40) + : +- Sort (48) + : +- Exchange (47) + : +- Project (46) + : +- Filter (45) + : +- Scan parquet (44) + +- Sort (57) + +- Filter (56) + +- HashAggregate (55) + +- Exchange (54) + +- HashAggregate (53) + +- Filter (52) + +- Scan parquet (51) + + +(1) ScanTransformer parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] @@ -96,7 +95,7 @@ Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] (9) InputIteratorTransformer Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -143,7 +142,7 @@ Join condition: None Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(21) Scan parquet +(21) ScanTransformer parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -215,141 +214,134 @@ Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity) (36) RegularHashAggregateExecTransformer Input [1]: [l_extendedprice#X] Keys: [] -Functions [1]: [partial_sum(l_extendedprice#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(37) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [sum(l_extendedprice#X)#X] -(38) ProjectExecTransformer +(37) ProjectExecTransformer Output [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] Input [1]: [sum(l_extendedprice#X)#X] -(39) WholeStageCodegenTransformer (X) +(38) WholeStageCodegenTransformer (X) Input [1]: [avg_yearly#X] Arguments: false -(40) VeloxColumnarToRow +(39) VeloxColumnarToRow Input [1]: [avg_yearly#X] -(41) Scan parquet +(40) Scan parquet Output [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey), IsNotNull(l_quantity)] ReadSchema: struct -(42) Filter +(41) Filter Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Condition : (isnotnull(l_partkey#X) AND isnotnull(l_quantity#X)) -(43) Exchange +(42) Exchange Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(44) Sort +(43) Sort Input [3]: [l_partkey#X, l_quantity#X, l_extendedprice#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(45) Scan parquet +(44) Scan parquet Output [3]: [p_partkey#X, p_brand#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_brand), IsNotNull(p_container), EqualTo(p_brand,Brand#X), EqualTo(p_container,MED BOX), IsNotNull(p_partkey)] ReadSchema: struct -(46) Filter +(45) Filter Input [3]: [p_partkey#X, p_brand#X, p_container#X] Condition : ((((isnotnull(p_brand#X) AND isnotnull(p_container#X)) AND (p_brand#X = Brand#X)) AND (p_container#X = MED BOX)) AND isnotnull(p_partkey#X)) -(47) Project +(46) Project Output [1]: [p_partkey#X] Input [3]: [p_partkey#X, p_brand#X, p_container#X] -(48) Exchange +(47) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(49) Sort +(48) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(50) SortMergeJoin +(49) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: None -(51) Project +(50) Project Output [3]: [l_quantity#X, l_extendedprice#X, p_partkey#X] Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, p_partkey#X] -(52) Scan parquet +(51) Scan parquet Output [2]: [l_partkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_partkey)] ReadSchema: struct -(53) Filter +(52) Filter Input [2]: [l_partkey#X, l_quantity#X] Condition : isnotnull(l_partkey#X) -(54) HashAggregate +(53) HashAggregate Input [2]: [l_partkey#X, l_quantity#X] Keys [1]: [l_partkey#X] Functions [1]: [partial_avg(l_quantity#X)] Aggregate Attributes [2]: [sum#X, count#X] Results [3]: [l_partkey#X, sum#X, count#X] -(55) Exchange +(54) Exchange Input [3]: [l_partkey#X, sum#X, count#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) HashAggregate +(55) HashAggregate Input [3]: [l_partkey#X, sum#X, count#X] Keys [1]: [l_partkey#X] Functions [1]: [avg(l_quantity#X)] Aggregate Attributes [1]: [avg(l_quantity#X)#X] Results [2]: [(0.2 * avg(l_quantity#X)#X) AS (0.2 * avg(l_quantity))#X, l_partkey#X] -(57) Filter +(56) Filter Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Condition : isnotnull((0.2 * avg(l_quantity))#X) -(58) Sort +(57) Sort Input [2]: [(0.2 * avg(l_quantity))#X, l_partkey#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(59) SortMergeJoin +(58) SortMergeJoin Left keys [1]: [p_partkey#X] Right keys [1]: [l_partkey#X] Join type: Inner Join condition: (cast(l_quantity#X as decimal(18,7)) < (0.2 * avg(l_quantity))#X) -(60) Project +(59) Project Output [1]: [l_extendedprice#X] Input [5]: [l_quantity#X, l_extendedprice#X, p_partkey#X, (0.2 * avg(l_quantity))#X, l_partkey#X] -(61) HashAggregate +(60) HashAggregate Input [1]: [l_extendedprice#X] Keys: [] Functions [1]: [partial_sum(l_extendedprice#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(62) HashAggregate +(61) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum(l_extendedprice#X)] Aggregate Attributes [1]: [sum(l_extendedprice#X)#X] Results [1]: [(sum(l_extendedprice#X)#X / 7.0) AS avg_yearly#X] -(63) AdaptiveSparkPlan +(62) AdaptiveSparkPlan Output [1]: [avg_yearly#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt index 85b11fa8ca18..c1a6b181bb34 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/18.txt @@ -1,105 +1,104 @@ == Physical Plan == -AdaptiveSparkPlan (110) +AdaptiveSparkPlan (109) +- == Final Plan == - VeloxColumnarToRow (70) - +- TakeOrderedAndProjectExecTransformer (69) - +- ^ RegularHashAggregateExecTransformer (67) - +- ^ RegularHashAggregateExecTransformer (66) - +- ^ ProjectExecTransformer (65) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) - :- ^ InputIteratorTransformer (46) - : +- ShuffleQueryStage (44), Statistics(X) - : +- ColumnarExchange (43) - : +- VeloxResizeBatches (42) - : +- ^ ProjectExecTransformer (40) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (38) - : +- ShuffleQueryStage (36), Statistics(X) - : +- ColumnarExchange (35) - : +- VeloxResizeBatches (34) - : +- ^ ProjectExecTransformer (32) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) - : :- ^ InputIteratorTransformer (18) - : : +- ShuffleQueryStage (16), Statistics(X) - : : +- ColumnarExchange (15) - : : +- VeloxResizeBatches (14) - : : +- ^ ProjectExecTransformer (12) - : : +- ^ FilterExecTransformer (11) - : : +- ^ Scan parquet (10) - : +- ^ ProjectExecTransformer (30) - : +- ^ FilterExecTransformer (29) - : +- ^ RegularHashAggregateExecTransformer (28) - : +- ^ InputIteratorTransformer (27) - : +- ShuffleQueryStage (25), Statistics(X) - : +- ColumnarExchange (24) - : +- VeloxResizeBatches (23) - : +- ^ ProjectExecTransformer (21) - : +- ^ FlushableHashAggregateExecTransformer (20) - : +- ^ Scan parquet (19) - +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) - :- ^ InputIteratorTransformer (55) - : +- ShuffleQueryStage (53), Statistics(X) - : +- ColumnarExchange (52) - : +- VeloxResizeBatches (51) - : +- ^ ProjectExecTransformer (49) - : +- ^ FilterExecTransformer (48) - : +- ^ Scan parquet (47) - +- ^ ProjectExecTransformer (62) - +- ^ FilterExecTransformer (61) - +- ^ RegularHashAggregateExecTransformer (60) - +- ^ InputIteratorTransformer (59) - +- ShuffleQueryStage (57), Statistics(X) - +- ReusedExchange (56) + VeloxColumnarToRow (69) + +- TakeOrderedAndProjectExecTransformer (68) + +- ^ RegularHashAggregateExecTransformer (66) + +- ^ ProjectExecTransformer (65) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (64) + :- ^ InputIteratorTransformer (46) + : +- ShuffleQueryStage (44), Statistics(X) + : +- ColumnarExchange (43) + : +- VeloxResizeBatches (42) + : +- ^ ProjectExecTransformer (40) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (39) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (38) + : +- ShuffleQueryStage (36), Statistics(X) + : +- ColumnarExchange (35) + : +- VeloxResizeBatches (34) + : +- ^ ProjectExecTransformer (32) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (31) + : :- ^ InputIteratorTransformer (18) + : : +- ShuffleQueryStage (16), Statistics(X) + : : +- ColumnarExchange (15) + : : +- VeloxResizeBatches (14) + : : +- ^ ProjectExecTransformer (12) + : : +- ^ FilterExecTransformer (11) + : : +- ^ ScanTransformer parquet (10) + : +- ^ ProjectExecTransformer (30) + : +- ^ FilterExecTransformer (29) + : +- ^ RegularHashAggregateExecTransformer (28) + : +- ^ InputIteratorTransformer (27) + : +- ShuffleQueryStage (25), Statistics(X) + : +- ColumnarExchange (24) + : +- VeloxResizeBatches (23) + : +- ^ ProjectExecTransformer (21) + : +- ^ FlushableHashAggregateExecTransformer (20) + : +- ^ ScanTransformer parquet (19) + +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (63) + :- ^ InputIteratorTransformer (55) + : +- ShuffleQueryStage (53), Statistics(X) + : +- ColumnarExchange (52) + : +- VeloxResizeBatches (51) + : +- ^ ProjectExecTransformer (49) + : +- ^ FilterExecTransformer (48) + : +- ^ ScanTransformer parquet (47) + +- ^ ProjectExecTransformer (62) + +- ^ FilterExecTransformer (61) + +- ^ RegularHashAggregateExecTransformer (60) + +- ^ InputIteratorTransformer (59) + +- ShuffleQueryStage (57), Statistics(X) + +- ReusedExchange (56) +- == Initial Plan == - TakeOrderedAndProject (109) - +- HashAggregate (108) - +- HashAggregate (107) - +- Project (106) - +- SortMergeJoin Inner (105) - :- Sort (92) - : +- Exchange (91) - : +- Project (90) - : +- SortMergeJoin Inner (89) - : :- Sort (74) - : : +- Exchange (73) - : : +- Filter (72) - : : +- Scan parquet (71) - : +- Sort (88) - : +- Exchange (87) - : +- SortMergeJoin LeftSemi (86) - : :- Sort (78) - : : +- Exchange (77) - : : +- Filter (76) - : : +- Scan parquet (75) - : +- Sort (85) - : +- Project (84) - : +- Filter (83) - : +- HashAggregate (82) - : +- Exchange (81) - : +- HashAggregate (80) - : +- Scan parquet (79) - +- SortMergeJoin LeftSemi (104) - :- Sort (96) - : +- Exchange (95) - : +- Filter (94) - : +- Scan parquet (93) - +- Sort (103) - +- Project (102) - +- Filter (101) - +- HashAggregate (100) - +- Exchange (99) - +- HashAggregate (98) - +- Scan parquet (97) - - -(1) Scan parquet + TakeOrderedAndProject (108) + +- HashAggregate (107) + +- HashAggregate (106) + +- Project (105) + +- SortMergeJoin Inner (104) + :- Sort (91) + : +- Exchange (90) + : +- Project (89) + : +- SortMergeJoin Inner (88) + : :- Sort (73) + : : +- Exchange (72) + : : +- Filter (71) + : : +- Scan parquet (70) + : +- Sort (87) + : +- Exchange (86) + : +- SortMergeJoin LeftSemi (85) + : :- Sort (77) + : : +- Exchange (76) + : : +- Filter (75) + : : +- Scan parquet (74) + : +- Sort (84) + : +- Project (83) + : +- Filter (82) + : +- HashAggregate (81) + : +- Exchange (80) + : +- HashAggregate (79) + : +- Scan parquet (78) + +- SortMergeJoin LeftSemi (103) + :- Sort (95) + : +- Exchange (94) + : +- Filter (93) + : +- Scan parquet (92) + +- Sort (102) + +- Project (101) + +- Filter (100) + +- HashAggregate (99) + +- Exchange (98) + +- HashAggregate (97) + +- Scan parquet (96) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -136,7 +135,7 @@ Input [2]: [c_custkey#X, c_name#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_name#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -173,7 +172,7 @@ Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] (18) InputIteratorTransformer Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -291,7 +290,7 @@ Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] (46) InputIteratorTransformer Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] -(47) Scan parquet +(47) ScanTransformer parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] @@ -375,223 +374,216 @@ Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, (66) RegularHashAggregateExecTransformer Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] - -(67) RegularHashAggregateExecTransformer -Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(68) WholeStageCodegenTransformer (X) +(67) WholeStageCodegenTransformer (X) Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: false -(69) TakeOrderedAndProjectExecTransformer +(68) TakeOrderedAndProjectExecTransformer Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X], 0 -(70) VeloxColumnarToRow +(69) VeloxColumnarToRow Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(71) Scan parquet +(70) Scan parquet Output [2]: [c_custkey#X, c_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_custkey)] ReadSchema: struct -(72) Filter +(71) Filter Input [2]: [c_custkey#X, c_name#X] Condition : isnotnull(c_custkey#X) -(73) Exchange +(72) Exchange Input [2]: [c_custkey#X, c_name#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(74) Sort +(73) Sort Input [2]: [c_custkey#X, c_name#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(75) Scan parquet +(74) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(76) Filter +(75) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Condition : (isnotnull(o_custkey#X) AND isnotnull(o_orderkey#X)) -(77) Exchange +(76) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(78) Sort +(77) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(79) Scan parquet +(78) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(80) HashAggregate +(79) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(81) Exchange +(80) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(82) HashAggregate +(81) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(83) Filter +(82) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(84) Project +(83) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(85) Sort +(84) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(86) SortMergeJoin +(85) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(87) Exchange +(86) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(88) Sort +(87) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(89) SortMergeJoin +(88) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(90) Project +(89) Project Output [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_custkey#X, o_totalprice#X, o_orderdate#X] -(91) Exchange +(90) Exchange Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(92) Sort +(91) Sort Input [5]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(93) Scan parquet +(92) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_orderkey)] ReadSchema: struct -(94) Filter +(93) Filter Input [2]: [l_orderkey#X, l_quantity#X] Condition : isnotnull(l_orderkey#X) -(95) Exchange +(94) Exchange Input [2]: [l_orderkey#X, l_quantity#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(96) Sort +(95) Sort Input [2]: [l_orderkey#X, l_quantity#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(97) Scan parquet +(96) Scan parquet Output [2]: [l_orderkey#X, l_quantity#X] Batched: true Location: InMemoryFileIndex [*] ReadSchema: struct -(98) HashAggregate +(97) HashAggregate Input [2]: [l_orderkey#X, l_quantity#X] Keys [1]: [l_orderkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [3]: [l_orderkey#X, sum#X, isEmpty#X] -(99) Exchange +(98) Exchange Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(100) HashAggregate +(99) HashAggregate Input [3]: [l_orderkey#X, sum#X, isEmpty#X] Keys [1]: [l_orderkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [2]: [l_orderkey#X, sum(l_quantity#X)#X AS sum(l_quantity#X)#X] -(101) Filter +(100) Filter Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] Condition : (isnotnull(sum(l_quantity#X)#X) AND (sum(l_quantity#X)#X > 300.00)) -(102) Project +(101) Project Output [1]: [l_orderkey#X] Input [2]: [l_orderkey#X, sum(l_quantity#X)#X] -(103) Sort +(102) Sort Input [1]: [l_orderkey#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(104) SortMergeJoin +(103) SortMergeJoin Left keys [1]: [l_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: None -(105) SortMergeJoin +(104) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(106) Project +(105) Project Output [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Input [7]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_orderkey#X, l_quantity#X] -(107) HashAggregate +(106) HashAggregate Input [6]: [c_custkey#X, c_name#X, o_orderkey#X, o_totalprice#X, o_orderdate#X, l_quantity#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] -(108) HashAggregate +(107) HashAggregate Input [7]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum#X, isEmpty#X] Keys [5]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity#X)#X AS sum(l_quantity)#X] -(109) TakeOrderedAndProject +(108) TakeOrderedAndProject Input [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: X, [o_totalprice#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] -(110) AdaptiveSparkPlan +(109) AdaptiveSparkPlan Output [6]: [c_name#X, c_custkey#X, o_orderkey#X, o_orderdate#X, o_totalprice#X, sum(l_quantity)#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt index c72ca859ce37..4db9ca0c8393 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/19.txt @@ -1,42 +1,41 @@ == Physical Plan == -AdaptiveSparkPlan (38) +AdaptiveSparkPlan (37) +- == Final Plan == - VeloxColumnarToRow (24) - +- ^ RegularHashAggregateExecTransformer (22) - +- ^ RegularHashAggregateExecTransformer (21) - +- ^ ProjectExecTransformer (20) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) - :- ^ InputIteratorTransformer (9) - : +- ShuffleQueryStage (7), Statistics(X) - : +- ColumnarExchange (6) - : +- VeloxResizeBatches (5) - : +- ^ ProjectExecTransformer (3) - : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) - +- ^ InputIteratorTransformer (18) - +- ShuffleQueryStage (16), Statistics(X) - +- ColumnarExchange (15) - +- VeloxResizeBatches (14) - +- ^ ProjectExecTransformer (12) - +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + VeloxColumnarToRow (23) + +- ^ RegularHashAggregateExecTransformer (21) + +- ^ ProjectExecTransformer (20) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (19) + :- ^ InputIteratorTransformer (9) + : +- ShuffleQueryStage (7), Statistics(X) + : +- ColumnarExchange (6) + : +- VeloxResizeBatches (5) + : +- ^ ProjectExecTransformer (3) + : +- ^ FilterExecTransformer (2) + : +- ^ ScanTransformer parquet (1) + +- ^ InputIteratorTransformer (18) + +- ShuffleQueryStage (16), Statistics(X) + +- ColumnarExchange (15) + +- VeloxResizeBatches (14) + +- ^ ProjectExecTransformer (12) + +- ^ FilterExecTransformer (11) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == - HashAggregate (37) - +- HashAggregate (36) - +- Project (35) - +- SortMergeJoin Inner (34) - :- Sort (29) - : +- Exchange (28) - : +- Project (27) - : +- Filter (26) - : +- Scan parquet (25) - +- Sort (33) - +- Exchange (32) - +- Filter (31) - +- Scan parquet (30) - - -(1) Scan parquet + HashAggregate (36) + +- HashAggregate (35) + +- Project (34) + +- SortMergeJoin Inner (33) + :- Sort (28) + : +- Exchange (27) + : +- Project (26) + : +- Filter (25) + : +- Scan parquet (24) + +- Sort (32) + +- Exchange (31) + +- Filter (30) + +- Scan parquet (29) + + +(1) ScanTransformer parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] @@ -73,7 +72,7 @@ Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] (9) InputIteratorTransformer Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] @@ -123,90 +122,83 @@ Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partke (21) RegularHashAggregateExecTransformer Input [3]: [l_extendedprice#X, l_discount#X, _pre_X#X] Keys: [] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [2]: [sum#X, isEmpty#X] - -(22) RegularHashAggregateExecTransformer -Input [2]: [sum#X, isEmpty#X] -Keys: [] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(23) WholeStageCodegenTransformer (X) +(22) WholeStageCodegenTransformer (X) Input [1]: [revenue#X] Arguments: false -(24) VeloxColumnarToRow +(23) VeloxColumnarToRow Input [1]: [revenue#X] -(25) Scan parquet +(24) Scan parquet Output [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipinstruct), In(l_shipmode, [AIR,AIR REG]), EqualTo(l_shipinstruct,DELIVER IN PERSON), IsNotNull(l_partkey), Or(Or(And(GreaterThanOrEqual(l_quantity,1.00),LessThanOrEqual(l_quantity,11.00)),And(GreaterThanOrEqual(l_quantity,10.00),LessThanOrEqual(l_quantity,20.00))),And(GreaterThanOrEqual(l_quantity,20.00),LessThanOrEqual(l_quantity,30.00)))] ReadSchema: struct -(26) Filter +(25) Filter Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] Condition : ((((isnotnull(l_shipinstruct#X) AND l_shipmode#X IN (AIR,AIR REG)) AND (l_shipinstruct#X = DELIVER IN PERSON)) AND isnotnull(l_partkey#X)) AND ((((l_quantity#X >= 1.00) AND (l_quantity#X <= 11.00)) OR ((l_quantity#X >= 10.00) AND (l_quantity#X <= 20.00))) OR ((l_quantity#X >= 20.00) AND (l_quantity#X <= 30.00)))) -(27) Project +(26) Project Output [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Input [6]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, l_shipinstruct#X, l_shipmode#X] -(28) Exchange +(27) Exchange Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(29) Sort +(28) Sort Input [4]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(30) Scan parquet +(29) Scan parquet Output [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_size), GreaterThanOrEqual(p_size,1), IsNotNull(p_partkey), Or(Or(And(And(EqualTo(p_brand,Brand#X),In(p_container, [SM BOX,SM CASE,SM PACK,SM PKG])),LessThanOrEqual(p_size,5)),And(And(EqualTo(p_brand,Brand#X),In(p_container, [MED BAG,MED BOX,MED PACK,MED PKG])),LessThanOrEqual(p_size,10))),And(And(EqualTo(p_brand,Brand#X),In(p_container, [LG BOX,LG CASE,LG PACK,LG PKG])),LessThanOrEqual(p_size,15)))] ReadSchema: struct -(31) Filter +(30) Filter Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Condition : (((isnotnull(p_size#X) AND (p_size#X >= 1)) AND isnotnull(p_partkey#X)) AND (((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (p_size#X <= 5)) OR (((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (p_size#X <= 10))) OR (((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (p_size#X <= 15)))) -(32) Exchange +(31) Exchange Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(33) Sort +(32) Sort Input [4]: [p_partkey#X, p_brand#X, p_size#X, p_container#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(34) SortMergeJoin +(33) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: Inner Join condition: (((((((p_brand#X = Brand#X) AND p_container#X IN (SM CASE,SM BOX,SM PACK,SM PKG)) AND (l_quantity#X >= 1.00)) AND (l_quantity#X <= 11.00)) AND (p_size#X <= 5)) OR (((((p_brand#X = Brand#X) AND p_container#X IN (MED BAG,MED BOX,MED PKG,MED PACK)) AND (l_quantity#X >= 10.00)) AND (l_quantity#X <= 20.00)) AND (p_size#X <= 10))) OR (((((p_brand#X = Brand#X) AND p_container#X IN (LG CASE,LG BOX,LG PACK,LG PKG)) AND (l_quantity#X >= 20.00)) AND (l_quantity#X <= 30.00)) AND (p_size#X <= 15))) -(35) Project +(34) Project Output [2]: [l_extendedprice#X, l_discount#X] Input [8]: [l_partkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, p_partkey#X, p_brand#X, p_size#X, p_container#X] -(36) HashAggregate +(35) HashAggregate Input [2]: [l_extendedprice#X, l_discount#X] Keys: [] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [2]: [sum#X, isEmpty#X] -(37) HashAggregate +(36) HashAggregate Input [2]: [sum#X, isEmpty#X] Keys: [] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X] -(38) AdaptiveSparkPlan +(37) AdaptiveSparkPlan Output [1]: [revenue#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt index ed09215178a4..4d8eee1f1fb7 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/20.txt @@ -1,32 +1,32 @@ == Physical Plan == -AdaptiveSparkPlan (143) +AdaptiveSparkPlan (142) +- == Final Plan == - VeloxColumnarToRow (93) - +- AQEShuffleRead (92) - +- ShuffleQueryStage (91), Statistics(X) - +- ColumnarExchange (90) - +- VeloxResizeBatches (89) - +- ^ ProjectExecTransformer (87) - +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (86) - :- ^ InputIteratorTransformer (76) - : +- ShuffleQueryStage (74), Statistics(X) - : +- ColumnarExchange (73) - : +- VeloxResizeBatches (72) - : +- ^ ProjectExecTransformer (70) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (69) + VeloxColumnarToRow (92) + +- AQEShuffleRead (91) + +- ShuffleQueryStage (90), Statistics(X) + +- ColumnarExchange (89) + +- VeloxResizeBatches (88) + +- ^ ProjectExecTransformer (86) + +- ^ ShuffledHashJoinExecTransformer Inner BuildRight (85) + :- ^ InputIteratorTransformer (75) + : +- ShuffleQueryStage (73), Statistics(X) + : +- ColumnarExchange (72) + : +- VeloxResizeBatches (71) + : +- ^ ProjectExecTransformer (69) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (68) : :- ^ InputIteratorTransformer (9) : : +- ShuffleQueryStage (7), Statistics(X) : : +- ColumnarExchange (6) : : +- VeloxResizeBatches (5) : : +- ^ ProjectExecTransformer (3) : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (68) - : +- ShuffleQueryStage (66), Statistics(X) - : +- ColumnarExchange (65) - : +- VeloxResizeBatches (64) - : +- ^ ProjectExecTransformer (62) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (61) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (67) + : +- ShuffleQueryStage (65), Statistics(X) + : +- ColumnarExchange (64) + : +- VeloxResizeBatches (63) + : +- ^ ProjectExecTransformer (61) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (60) : :- ^ InputIteratorTransformer (35) : : +- ShuffleQueryStage (33), Statistics(X) : : +- ColumnarExchange (32) @@ -39,94 +39,93 @@ AdaptiveSparkPlan (143) : : : +- VeloxResizeBatches (14) : : : +- ^ ProjectExecTransformer (12) : : : +- ^ FilterExecTransformer (11) - : : : +- ^ Scan parquet (10) + : : : +- ^ ScanTransformer parquet (10) : : +- ^ InputIteratorTransformer (27) : : +- ShuffleQueryStage (25), Statistics(X) : : +- ColumnarExchange (24) : : +- VeloxResizeBatches (23) : : +- ^ ProjectExecTransformer (21) : : +- ^ FilterExecTransformer (20) - : : +- ^ Scan parquet (19) - : +- ^ InputIteratorTransformer (60) - : +- ShuffleQueryStage (58), Statistics(X) - : +- ColumnarExchange (57) - : +- VeloxResizeBatches (56) - : +- ^ ProjectExecTransformer (54) - : +- ^ FilterExecTransformer (53) - : +- ^ ProjectExecTransformer (52) - : +- ^ RegularHashAggregateExecTransformer (51) - : +- ^ RegularHashAggregateExecTransformer (50) - : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) - : :- ^ InputIteratorTransformer (44) - : : +- ShuffleQueryStage (42), Statistics(X) - : : +- ColumnarExchange (41) - : : +- VeloxResizeBatches (40) - : : +- ^ ProjectExecTransformer (38) - : : +- ^ FilterExecTransformer (37) - : : +- ^ Scan parquet (36) - : +- ^ InputIteratorTransformer (48) - : +- ShuffleQueryStage (46), Statistics(X) - : +- ReusedExchange (45) - +- ^ InputIteratorTransformer (85) - +- ShuffleQueryStage (83), Statistics(X) - +- ColumnarExchange (82) - +- VeloxResizeBatches (81) - +- ^ ProjectExecTransformer (79) - +- ^ FilterExecTransformer (78) - +- ^ Scan parquet (77) + : : +- ^ ScanTransformer parquet (19) + : +- ^ InputIteratorTransformer (59) + : +- ShuffleQueryStage (57), Statistics(X) + : +- ColumnarExchange (56) + : +- VeloxResizeBatches (55) + : +- ^ ProjectExecTransformer (53) + : +- ^ FilterExecTransformer (52) + : +- ^ ProjectExecTransformer (51) + : +- ^ RegularHashAggregateExecTransformer (50) + : +- ^ ShuffledHashJoinExecTransformer LeftSemi BuildRight (49) + : :- ^ InputIteratorTransformer (44) + : : +- ShuffleQueryStage (42), Statistics(X) + : : +- ColumnarExchange (41) + : : +- VeloxResizeBatches (40) + : : +- ^ ProjectExecTransformer (38) + : : +- ^ FilterExecTransformer (37) + : : +- ^ ScanTransformer parquet (36) + : +- ^ InputIteratorTransformer (48) + : +- ShuffleQueryStage (46), Statistics(X) + : +- ReusedExchange (45) + +- ^ InputIteratorTransformer (84) + +- ShuffleQueryStage (82), Statistics(X) + +- ColumnarExchange (81) + +- VeloxResizeBatches (80) + +- ^ ProjectExecTransformer (78) + +- ^ FilterExecTransformer (77) + +- ^ ScanTransformer parquet (76) +- == Initial Plan == - Sort (142) - +- Exchange (141) - +- Project (140) - +- SortMergeJoin Inner (139) - :- Sort (133) - : +- Exchange (132) - : +- Project (131) - : +- SortMergeJoin LeftSemi (130) - : :- Sort (97) - : : +- Exchange (96) - : : +- Filter (95) - : : +- Scan parquet (94) - : +- Sort (129) - : +- Exchange (128) - : +- Project (127) - : +- SortMergeJoin Inner (126) - : :- Sort (109) - : : +- Exchange (108) - : : +- SortMergeJoin LeftSemi (107) - : : :- Sort (101) - : : : +- Exchange (100) - : : : +- Filter (99) - : : : +- Scan parquet (98) - : : +- Sort (106) - : : +- Exchange (105) - : : +- Project (104) - : : +- Filter (103) - : : +- Scan parquet (102) - : +- Sort (125) - : +- Exchange (124) - : +- Filter (123) - : +- HashAggregate (122) - : +- HashAggregate (121) - : +- SortMergeJoin LeftSemi (120) - : :- Sort (114) - : : +- Exchange (113) - : : +- Project (112) - : : +- Filter (111) - : : +- Scan parquet (110) - : +- Sort (119) - : +- Exchange (118) - : +- Project (117) - : +- Filter (116) - : +- Scan parquet (115) - +- Sort (138) - +- Exchange (137) - +- Project (136) - +- Filter (135) - +- Scan parquet (134) - - -(1) Scan parquet + Sort (141) + +- Exchange (140) + +- Project (139) + +- SortMergeJoin Inner (138) + :- Sort (132) + : +- Exchange (131) + : +- Project (130) + : +- SortMergeJoin LeftSemi (129) + : :- Sort (96) + : : +- Exchange (95) + : : +- Filter (94) + : : +- Scan parquet (93) + : +- Sort (128) + : +- Exchange (127) + : +- Project (126) + : +- SortMergeJoin Inner (125) + : :- Sort (108) + : : +- Exchange (107) + : : +- SortMergeJoin LeftSemi (106) + : : :- Sort (100) + : : : +- Exchange (99) + : : : +- Filter (98) + : : : +- Scan parquet (97) + : : +- Sort (105) + : : +- Exchange (104) + : : +- Project (103) + : : +- Filter (102) + : : +- Scan parquet (101) + : +- Sort (124) + : +- Exchange (123) + : +- Filter (122) + : +- HashAggregate (121) + : +- HashAggregate (120) + : +- SortMergeJoin LeftSemi (119) + : :- Sort (113) + : : +- Exchange (112) + : : +- Project (111) + : : +- Filter (110) + : : +- Scan parquet (109) + : +- Sort (118) + : +- Exchange (117) + : +- Project (116) + : +- Filter (115) + : +- Scan parquet (114) + +- Sort (137) + +- Exchange (136) + +- Project (135) + +- Filter (134) + +- Scan parquet (133) + + +(1) ScanTransformer parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -163,7 +162,7 @@ Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] (9) InputIteratorTransformer Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] @@ -200,7 +199,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (18) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -269,7 +268,7 @@ Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] (35) InputIteratorTransformer Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] -(36) Scan parquet +(36) ScanTransformer parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -328,415 +327,408 @@ Join condition: None (50) RegularHashAggregateExecTransformer Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] -Functions [1]: [partial_sum(l_quantity#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] - -(51) RegularHashAggregateExecTransformer -Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(52) ProjectExecTransformer +(51) ProjectExecTransformer Output [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [l_partkey#X, l_suppkey#X, sum(l_quantity#X)#X] -(53) FilterExecTransformer +(52) FilterExecTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: isnotnull((0.5 * sum(l_quantity))#X) -(54) ProjectExecTransformer +(53) ProjectExecTransformer Output [4]: [hash(l_partkey#X, l_suppkey#X, 42) AS hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(55) WholeStageCodegenTransformer (X) +(54) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: false -(56) VeloxResizeBatches +(55) VeloxResizeBatches Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X, X -(57) ColumnarExchange +(56) ColumnarExchange Input [4]: [hash_partition_key#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(58) ShuffleQueryStage +(57) ShuffleQueryStage Output [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: X -(59) InputAdapter +(58) InputAdapter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(60) InputIteratorTransformer +(59) InputIteratorTransformer Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(61) ShuffledHashJoinExecTransformer +(60) ShuffledHashJoinExecTransformer Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(62) ProjectExecTransformer +(61) ProjectExecTransformer Output [2]: [hash(ps_suppkey#X, 42) AS hash_partition_key#X, ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(63) WholeStageCodegenTransformer (X) +(62) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: false -(64) VeloxResizeBatches +(63) VeloxResizeBatches Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: X, X -(65) ColumnarExchange +(64) ColumnarExchange Input [2]: [hash_partition_key#X, ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [ps_suppkey#X], [plan_id=X], [shuffle_writer_type=hash] -(66) ShuffleQueryStage +(65) ShuffleQueryStage Output [1]: [ps_suppkey#X] Arguments: X -(67) InputAdapter +(66) InputAdapter Input [1]: [ps_suppkey#X] -(68) InputIteratorTransformer +(67) InputIteratorTransformer Input [1]: [ps_suppkey#X] -(69) ShuffledHashJoinExecTransformer +(68) ShuffledHashJoinExecTransformer Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(70) ProjectExecTransformer +(69) ProjectExecTransformer Output [4]: [hash(s_nationkey#X, 42) AS hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(71) WholeStageCodegenTransformer (X) +(70) WholeStageCodegenTransformer (X) Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: false -(72) VeloxResizeBatches +(71) VeloxResizeBatches Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: X, X -(73) ColumnarExchange +(72) ColumnarExchange Input [4]: [hash_partition_key#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [s_name#X, s_address#X, s_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(74) ShuffleQueryStage +(73) ShuffleQueryStage Output [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: X -(75) InputAdapter +(74) InputAdapter Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(76) InputIteratorTransformer +(75) InputIteratorTransformer Input [3]: [s_name#X, s_address#X, s_nationkey#X] -(77) Scan parquet +(76) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(78) FilterExecTransformer +(77) FilterExecTransformer Input [2]: [n_nationkey#X, n_name#X] Arguments: ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(79) ProjectExecTransformer +(78) ProjectExecTransformer Output [2]: [hash(n_nationkey#X, 42) AS hash_partition_key#X, n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(80) WholeStageCodegenTransformer (X) +(79) WholeStageCodegenTransformer (X) Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: false -(81) VeloxResizeBatches +(80) VeloxResizeBatches Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: X, X -(82) ColumnarExchange +(81) ColumnarExchange Input [2]: [hash_partition_key#X, n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [n_nationkey#X], [plan_id=X], [shuffle_writer_type=hash] -(83) ShuffleQueryStage +(82) ShuffleQueryStage Output [1]: [n_nationkey#X] Arguments: X -(84) InputAdapter +(83) InputAdapter Input [1]: [n_nationkey#X] -(85) InputIteratorTransformer +(84) InputIteratorTransformer Input [1]: [n_nationkey#X] -(86) ShuffledHashJoinExecTransformer +(85) ShuffledHashJoinExecTransformer Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(87) ProjectExecTransformer +(86) ProjectExecTransformer Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(88) WholeStageCodegenTransformer (X) +(87) WholeStageCodegenTransformer (X) Input [2]: [s_name#X, s_address#X] Arguments: false -(89) VeloxResizeBatches +(88) VeloxResizeBatches Input [2]: [s_name#X, s_address#X] Arguments: X, X -(90) ColumnarExchange +(89) ColumnarExchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X], [shuffle_writer_type=hash] -(91) ShuffleQueryStage +(90) ShuffleQueryStage Output [2]: [s_name#X, s_address#X] Arguments: X -(92) AQEShuffleRead +(91) AQEShuffleRead Input [2]: [s_name#X, s_address#X] Arguments: local -(93) VeloxColumnarToRow +(92) VeloxColumnarToRow Input [2]: [s_name#X, s_address#X] -(94) Scan parquet +(93) Scan parquet Output [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(s_nationkey)] ReadSchema: struct -(95) Filter +(94) Filter Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Condition : isnotnull(s_nationkey#X) -(96) Exchange +(95) Exchange Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(97) Sort +(96) Sort Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] Arguments: [s_suppkey#X ASC NULLS FIRST], false, 0 -(98) Scan parquet +(97) Scan parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(ps_availqty), IsNotNull(ps_partkey), IsNotNull(ps_suppkey)] ReadSchema: struct -(99) Filter +(98) Filter Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Condition : ((isnotnull(ps_availqty#X) AND isnotnull(ps_partkey#X)) AND isnotnull(ps_suppkey#X)) -(100) Exchange +(99) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(101) Sort +(100) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST], false, 0 -(102) Scan parquet +(101) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(103) Filter +(102) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(104) Project +(103) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(105) Exchange +(104) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(106) Sort +(105) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(107) SortMergeJoin +(106) SortMergeJoin Left keys [1]: [ps_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(108) Exchange +(107) Exchange Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: hashpartitioning(ps_partkey#X, ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(109) Sort +(108) Sort Input [3]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X] Arguments: [ps_partkey#X ASC NULLS FIRST, ps_suppkey#X ASC NULLS FIRST], false, 0 -(110) Scan parquet +(109) Scan parquet Output [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThanOrEqual(l_shipdate,1994-01-01), LessThan(l_shipdate,1995-01-01), IsNotNull(l_partkey), IsNotNull(l_suppkey)] ReadSchema: struct -(111) Filter +(110) Filter Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] Condition : ((((isnotnull(l_shipdate#X) AND (l_shipdate#X >= 1994-01-01)) AND (l_shipdate#X < 1995-01-01)) AND isnotnull(l_partkey#X)) AND isnotnull(l_suppkey#X)) -(112) Project +(111) Project Output [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Input [4]: [l_partkey#X, l_suppkey#X, l_quantity#X, l_shipdate#X] -(113) Exchange +(112) Exchange Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: hashpartitioning(l_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(114) Sort +(113) Sort Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Arguments: [l_partkey#X ASC NULLS FIRST], false, 0 -(115) Scan parquet +(114) Scan parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(p_name), StringStartsWith(p_name,forest)] ReadSchema: struct -(116) Filter +(115) Filter Input [2]: [p_partkey#X, p_name#X] Condition : (isnotnull(p_name#X) AND StartsWith(p_name#X, forest)) -(117) Project +(116) Project Output [1]: [p_partkey#X] Input [2]: [p_partkey#X, p_name#X] -(118) Exchange +(117) Exchange Input [1]: [p_partkey#X] Arguments: hashpartitioning(p_partkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(119) Sort +(118) Sort Input [1]: [p_partkey#X] Arguments: [p_partkey#X ASC NULLS FIRST], false, 0 -(120) SortMergeJoin +(119) SortMergeJoin Left keys [1]: [l_partkey#X] Right keys [1]: [p_partkey#X] Join type: LeftSemi Join condition: None -(121) HashAggregate +(120) HashAggregate Input [3]: [l_partkey#X, l_suppkey#X, l_quantity#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [partial_sum(l_quantity#X)] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] -(122) HashAggregate +(121) HashAggregate Input [4]: [l_partkey#X, l_suppkey#X, sum#X, isEmpty#X] Keys [2]: [l_partkey#X, l_suppkey#X] Functions [1]: [sum(l_quantity#X)] Aggregate Attributes [1]: [sum(l_quantity#X)#X] Results [3]: [(0.5 * sum(l_quantity#X)#X) AS (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(123) Filter +(122) Filter Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Condition : isnotnull((0.5 * sum(l_quantity))#X) -(124) Exchange +(123) Exchange Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: hashpartitioning(l_partkey#X, l_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(125) Sort +(124) Sort Input [3]: [(0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] Arguments: [l_partkey#X ASC NULLS FIRST, l_suppkey#X ASC NULLS FIRST], false, 0 -(126) SortMergeJoin +(125) SortMergeJoin Left keys [2]: [ps_partkey#X, ps_suppkey#X] Right keys [2]: [l_partkey#X, l_suppkey#X] Join type: Inner Join condition: (cast(ps_availqty#X as decimal(24,3)) > (0.5 * sum(l_quantity))#X) -(127) Project +(126) Project Output [1]: [ps_suppkey#X] Input [6]: [ps_partkey#X, ps_suppkey#X, ps_availqty#X, (0.5 * sum(l_quantity))#X, l_partkey#X, l_suppkey#X] -(128) Exchange +(127) Exchange Input [1]: [ps_suppkey#X] Arguments: hashpartitioning(ps_suppkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(129) Sort +(128) Sort Input [1]: [ps_suppkey#X] Arguments: [ps_suppkey#X ASC NULLS FIRST], false, 0 -(130) SortMergeJoin +(129) SortMergeJoin Left keys [1]: [s_suppkey#X] Right keys [1]: [ps_suppkey#X] Join type: LeftSemi Join condition: None -(131) Project +(130) Project Output [3]: [s_name#X, s_address#X, s_nationkey#X] Input [4]: [s_suppkey#X, s_name#X, s_address#X, s_nationkey#X] -(132) Exchange +(131) Exchange Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: hashpartitioning(s_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(133) Sort +(132) Sort Input [3]: [s_name#X, s_address#X, s_nationkey#X] Arguments: [s_nationkey#X ASC NULLS FIRST], false, 0 -(134) Scan parquet +(133) Scan parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(n_name), EqualTo(n_name,CANADA), IsNotNull(n_nationkey)] ReadSchema: struct -(135) Filter +(134) Filter Input [2]: [n_nationkey#X, n_name#X] Condition : ((isnotnull(n_name#X) AND (n_name#X = CANADA)) AND isnotnull(n_nationkey#X)) -(136) Project +(135) Project Output [1]: [n_nationkey#X] Input [2]: [n_nationkey#X, n_name#X] -(137) Exchange +(136) Exchange Input [1]: [n_nationkey#X] Arguments: hashpartitioning(n_nationkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(138) Sort +(137) Sort Input [1]: [n_nationkey#X] Arguments: [n_nationkey#X ASC NULLS FIRST], false, 0 -(139) SortMergeJoin +(138) SortMergeJoin Left keys [1]: [s_nationkey#X] Right keys [1]: [n_nationkey#X] Join type: Inner Join condition: None -(140) Project +(139) Project Output [2]: [s_name#X, s_address#X] Input [4]: [s_name#X, s_address#X, s_nationkey#X, n_nationkey#X] -(141) Exchange +(140) Exchange Input [2]: [s_name#X, s_address#X] Arguments: rangepartitioning(s_name#X ASC NULLS FIRST, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(142) Sort +(141) Sort Input [2]: [s_name#X, s_address#X] Arguments: [s_name#X ASC NULLS FIRST], true, 0 -(143) AdaptiveSparkPlan +(142) AdaptiveSparkPlan Output [2]: [s_name#X, s_address#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt index 271d0c6d5098..96750b2b3ecc 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/21.txt @@ -29,7 +29,7 @@ AdaptiveSparkPlan (137) : : : +- VeloxResizeBatches (5) : : : +- ^ ProjectExecTransformer (3) : : : +- ^ FilterExecTransformer (2) - : : : +- ^ Scan parquet (1) + : : : +- ^ ScanTransformer parquet (1) : : +- ^ InputIteratorTransformer (44) : : +- ShuffleQueryStage (42), Statistics(X) : : +- ColumnarExchange (41) @@ -43,34 +43,34 @@ AdaptiveSparkPlan (137) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (26) : : : +- ShuffleQueryStage (24), Statistics(X) : : : +- ColumnarExchange (23) : : : +- VeloxResizeBatches (22) : : : +- ^ ProjectExecTransformer (20) - : : : +- ^ Scan parquet (19) + : : : +- ^ ScanTransformer parquet (19) : : +- ^ InputIteratorTransformer (36) : : +- ShuffleQueryStage (34), Statistics(X) : : +- ColumnarExchange (33) : : +- VeloxResizeBatches (32) : : +- ^ ProjectExecTransformer (30) : : +- ^ FilterExecTransformer (29) - : : +- ^ Scan parquet (28) + : : +- ^ ScanTransformer parquet (28) : +- ^ InputIteratorTransformer (61) : +- ShuffleQueryStage (59), Statistics(X) : +- ColumnarExchange (58) : +- VeloxResizeBatches (57) : +- ^ ProjectExecTransformer (55) : +- ^ FilterExecTransformer (54) - : +- ^ Scan parquet (53) + : +- ^ ScanTransformer parquet (53) +- ^ InputIteratorTransformer (78) +- ShuffleQueryStage (76), Statistics(X) +- ColumnarExchange (75) +- VeloxResizeBatches (74) +- ^ ProjectExecTransformer (72) +- ^ FilterExecTransformer (71) - +- ^ Scan parquet (70) + +- ^ ScanTransformer parquet (70) +- == Initial Plan == TakeOrderedAndProject (136) +- HashAggregate (135) @@ -119,7 +119,7 @@ AdaptiveSparkPlan (137) +- Scan parquet (126) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [s_suppkey#X, s_name#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -156,7 +156,7 @@ Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] (9) InputIteratorTransformer Input [3]: [s_suppkey#X, s_name#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -193,7 +193,7 @@ Input [2]: [l_orderkey#X, l_suppkey#X] (18) InputIteratorTransformer Input [2]: [l_orderkey#X, l_suppkey#X] -(19) Scan parquet +(19) ScanTransformer parquet Output [2]: [l_orderkey#X, l_suppkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -231,7 +231,7 @@ Right keys [1]: [l_orderkey#X] Join type: LeftSemi Join condition: NOT (l_suppkey#X = l_suppkey#X) -(28) Scan parquet +(28) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -332,7 +332,7 @@ Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] (52) InputIteratorTransformer Input [3]: [s_name#X, s_nationkey#X, l_orderkey#X] -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderstatus#X] Batched: true Location: InMemoryFileIndex [*] @@ -401,7 +401,7 @@ Input [2]: [s_name#X, s_nationkey#X] (69) InputIteratorTransformer Input [2]: [s_name#X, s_nationkey#X] -(70) Scan parquet +(70) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt index b8123fb6dc0e..c6e147da04d8 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/22.txt @@ -22,13 +22,13 @@ AdaptiveSparkPlan (52) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (17) +- ShuffleQueryStage (15), Statistics(X) +- ColumnarExchange (14) +- VeloxResizeBatches (13) +- ^ ProjectExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (51) +- Exchange (50) @@ -46,7 +46,7 @@ AdaptiveSparkPlan (52) +- Scan parquet (42) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [c_custkey#X, c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -83,7 +83,7 @@ Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] (9) InputIteratorTransformer Input [3]: [c_custkey#X, c_phone#X, c_acctbal#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [1]: [o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -285,7 +285,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) @@ -295,7 +295,7 @@ AdaptiveSparkPlan (72) +- Scan parquet (66) -(53) Scan parquet +(53) ScanTransformer parquet Output [2]: [c_phone#X, c_acctbal#X] Batched: true Location: InMemoryFileIndex [*] @@ -402,7 +402,7 @@ AdaptiveSparkPlan (72) +- ^ FlushableHashAggregateExecTransformer (56) +- ^ ProjectExecTransformer (55) +- ^ FilterExecTransformer (54) - +- ^ Scan parquet (53) + +- ^ ScanTransformer parquet (53) +- == Initial Plan == HashAggregate (71) +- Exchange (70) diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt index 8eea7581a417..1f86369a62af 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/3.txt @@ -1,67 +1,66 @@ == Physical Plan == -AdaptiveSparkPlan (67) +AdaptiveSparkPlan (66) +- == Final Plan == - VeloxColumnarToRow (43) - +- TakeOrderedAndProjectExecTransformer (42) - +- ^ ProjectExecTransformer (40) - +- ^ RegularHashAggregateExecTransformer (39) - +- ^ RegularHashAggregateExecTransformer (38) - +- ^ ProjectExecTransformer (37) - +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) - :- ^ InputIteratorTransformer (26) - : +- ShuffleQueryStage (24), Statistics(X) - : +- ColumnarExchange (23) - : +- VeloxResizeBatches (22) - : +- ^ ProjectExecTransformer (20) - : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) - : :- ^ InputIteratorTransformer (9) - : : +- ShuffleQueryStage (7), Statistics(X) - : : +- ColumnarExchange (6) - : : +- VeloxResizeBatches (5) - : : +- ^ ProjectExecTransformer (3) - : : +- ^ FilterExecTransformer (2) - : : +- ^ Scan parquet (1) - : +- ^ InputIteratorTransformer (18) - : +- ShuffleQueryStage (16), Statistics(X) - : +- ColumnarExchange (15) - : +- VeloxResizeBatches (14) - : +- ^ ProjectExecTransformer (12) - : +- ^ FilterExecTransformer (11) - : +- ^ Scan parquet (10) - +- ^ InputIteratorTransformer (35) - +- ShuffleQueryStage (33), Statistics(X) - +- ColumnarExchange (32) - +- VeloxResizeBatches (31) - +- ^ ProjectExecTransformer (29) - +- ^ FilterExecTransformer (28) - +- ^ Scan parquet (27) + VeloxColumnarToRow (42) + +- TakeOrderedAndProjectExecTransformer (41) + +- ^ ProjectExecTransformer (39) + +- ^ RegularHashAggregateExecTransformer (38) + +- ^ ProjectExecTransformer (37) + +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (36) + :- ^ InputIteratorTransformer (26) + : +- ShuffleQueryStage (24), Statistics(X) + : +- ColumnarExchange (23) + : +- VeloxResizeBatches (22) + : +- ^ ProjectExecTransformer (20) + : +- ^ ShuffledHashJoinExecTransformer Inner BuildLeft (19) + : :- ^ InputIteratorTransformer (9) + : : +- ShuffleQueryStage (7), Statistics(X) + : : +- ColumnarExchange (6) + : : +- VeloxResizeBatches (5) + : : +- ^ ProjectExecTransformer (3) + : : +- ^ FilterExecTransformer (2) + : : +- ^ ScanTransformer parquet (1) + : +- ^ InputIteratorTransformer (18) + : +- ShuffleQueryStage (16), Statistics(X) + : +- ColumnarExchange (15) + : +- VeloxResizeBatches (14) + : +- ^ ProjectExecTransformer (12) + : +- ^ FilterExecTransformer (11) + : +- ^ ScanTransformer parquet (10) + +- ^ InputIteratorTransformer (35) + +- ShuffleQueryStage (33), Statistics(X) + +- ColumnarExchange (32) + +- VeloxResizeBatches (31) + +- ^ ProjectExecTransformer (29) + +- ^ FilterExecTransformer (28) + +- ^ ScanTransformer parquet (27) +- == Initial Plan == - TakeOrderedAndProject (66) - +- HashAggregate (65) - +- HashAggregate (64) - +- Project (63) - +- SortMergeJoin Inner (62) - :- Sort (56) - : +- Exchange (55) - : +- Project (54) - : +- SortMergeJoin Inner (53) - : :- Sort (48) - : : +- Exchange (47) - : : +- Project (46) - : : +- Filter (45) - : : +- Scan parquet (44) - : +- Sort (52) - : +- Exchange (51) - : +- Filter (50) - : +- Scan parquet (49) - +- Sort (61) - +- Exchange (60) - +- Project (59) - +- Filter (58) - +- Scan parquet (57) - - -(1) Scan parquet + TakeOrderedAndProject (65) + +- HashAggregate (64) + +- HashAggregate (63) + +- Project (62) + +- SortMergeJoin Inner (61) + :- Sort (55) + : +- Exchange (54) + : +- Project (53) + : +- SortMergeJoin Inner (52) + : :- Sort (47) + : : +- Exchange (46) + : : +- Project (45) + : : +- Filter (44) + : : +- Scan parquet (43) + : +- Sort (51) + : +- Exchange (50) + : +- Filter (49) + : +- Scan parquet (48) + +- Sort (60) + +- Exchange (59) + +- Project (58) + +- Filter (57) + +- Scan parquet (56) + + +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] @@ -98,7 +97,7 @@ Input [1]: [c_custkey#X] (9) InputIteratorTransformer Input [1]: [c_custkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -167,7 +166,7 @@ Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] (26) InputIteratorTransformer Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -217,143 +216,136 @@ Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_exten (38) RegularHashAggregateExecTransformer Input [6]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X, _pre_X#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [partial_sum(_pre_X#X)] -Aggregate Attributes [2]: [sum#X, isEmpty#X] -Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] - -(39) RegularHashAggregateExecTransformer -Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] -Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] +Functions [1]: [sum(_pre_X#X)] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(40) ProjectExecTransformer +(39) ProjectExecTransformer Output [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] Input [4]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X] -(41) WholeStageCodegenTransformer (X) +(40) WholeStageCodegenTransformer (X) Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: false -(42) TakeOrderedAndProjectExecTransformer +(41) TakeOrderedAndProjectExecTransformer Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X], 0 -(43) VeloxColumnarToRow +(42) VeloxColumnarToRow Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(44) Scan parquet +(43) Scan parquet Output [2]: [c_custkey#X, c_mktsegment#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(c_mktsegment), EqualTo(c_mktsegment,BUILDING), IsNotNull(c_custkey)] ReadSchema: struct -(45) Filter +(44) Filter Input [2]: [c_custkey#X, c_mktsegment#X] Condition : ((isnotnull(c_mktsegment#X) AND (c_mktsegment#X = BUILDING)) AND isnotnull(c_custkey#X)) -(46) Project +(45) Project Output [1]: [c_custkey#X] Input [2]: [c_custkey#X, c_mktsegment#X] -(47) Exchange +(46) Exchange Input [1]: [c_custkey#X] Arguments: hashpartitioning(c_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(48) Sort +(47) Sort Input [1]: [c_custkey#X] Arguments: [c_custkey#X ASC NULLS FIRST], false, 0 -(49) Scan parquet +(48) Scan parquet Output [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(o_orderdate), LessThan(o_orderdate,1995-03-15), IsNotNull(o_custkey), IsNotNull(o_orderkey)] ReadSchema: struct -(50) Filter +(49) Filter Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Condition : (((isnotnull(o_orderdate#X) AND (o_orderdate#X < 1995-03-15)) AND isnotnull(o_custkey#X)) AND isnotnull(o_orderkey#X)) -(51) Exchange +(50) Exchange Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_custkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(52) Sort +(51) Sort Input [4]: [o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_custkey#X ASC NULLS FIRST], false, 0 -(53) SortMergeJoin +(52) SortMergeJoin Left keys [1]: [c_custkey#X] Right keys [1]: [o_custkey#X] Join type: Inner Join condition: None -(54) Project +(53) Project Output [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Input [5]: [c_custkey#X, o_orderkey#X, o_custkey#X, o_orderdate#X, o_shippriority#X] -(55) Exchange +(54) Exchange Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: hashpartitioning(o_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(56) Sort +(55) Sort Input [3]: [o_orderkey#X, o_orderdate#X, o_shippriority#X] Arguments: [o_orderkey#X ASC NULLS FIRST], false, 0 -(57) Scan parquet +(56) Scan parquet Output [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] PushedFilters: [IsNotNull(l_shipdate), GreaterThan(l_shipdate,1995-03-15), IsNotNull(l_orderkey)] ReadSchema: struct -(58) Filter +(57) Filter Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Condition : ((isnotnull(l_shipdate#X) AND (l_shipdate#X > 1995-03-15)) AND isnotnull(l_orderkey#X)) -(59) Project +(58) Project Output [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(60) Exchange +(59) Exchange Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: hashpartitioning(l_orderkey#X, 1), ENSURE_REQUIREMENTS, [plan_id=X] -(61) Sort +(60) Sort Input [3]: [l_orderkey#X, l_extendedprice#X, l_discount#X] Arguments: [l_orderkey#X ASC NULLS FIRST], false, 0 -(62) SortMergeJoin +(61) SortMergeJoin Left keys [1]: [o_orderkey#X] Right keys [1]: [l_orderkey#X] Join type: Inner Join condition: None -(63) Project +(62) Project Output [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Input [6]: [o_orderkey#X, o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] -(64) HashAggregate +(63) HashAggregate Input [5]: [o_orderdate#X, o_shippriority#X, l_orderkey#X, l_extendedprice#X, l_discount#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [partial_sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [2]: [sum#X, isEmpty#X] Results [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] -(65) HashAggregate +(64) HashAggregate Input [5]: [l_orderkey#X, o_orderdate#X, o_shippriority#X, sum#X, isEmpty#X] Keys [3]: [l_orderkey#X, o_orderdate#X, o_shippriority#X] Functions [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))] Aggregate Attributes [1]: [sum((l_extendedprice#X * (1 - l_discount#X)))#X] Results [4]: [l_orderkey#X, sum((l_extendedprice#X * (1 - l_discount#X)))#X AS revenue#X, o_orderdate#X, o_shippriority#X] -(66) TakeOrderedAndProject +(65) TakeOrderedAndProject Input [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: X, [revenue#X DESC NULLS LAST, o_orderdate#X ASC NULLS FIRST], [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] -(67) AdaptiveSparkPlan +(66) AdaptiveSparkPlan Output [4]: [l_orderkey#X, revenue#X, o_orderdate#X, o_shippriority#X] Arguments: isFinalPlan=true \ No newline at end of file diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt index 286881fb99b1..71a8f6974ed1 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/4.txt @@ -22,14 +22,14 @@ AdaptiveSparkPlan (56) : +- VeloxResizeBatches (5) : +- ^ ProjectExecTransformer (3) : +- ^ FilterExecTransformer (2) - : +- ^ Scan parquet (1) + : +- ^ ScanTransformer parquet (1) +- ^ InputIteratorTransformer (18) +- ShuffleQueryStage (16), Statistics(X) +- ColumnarExchange (15) +- VeloxResizeBatches (14) +- ^ ProjectExecTransformer (12) +- ^ FilterExecTransformer (11) - +- ^ Scan parquet (10) + +- ^ ScanTransformer parquet (10) +- == Initial Plan == Sort (55) +- Exchange (54) @@ -50,7 +50,7 @@ AdaptiveSparkPlan (56) +- Scan parquet (44) -(1) Scan parquet +(1) ScanTransformer parquet Output [3]: [o_orderkey#X, o_orderdate#X, o_orderpriority#X] Batched: true Location: InMemoryFileIndex [*] @@ -87,7 +87,7 @@ Input [2]: [o_orderkey#X, o_orderpriority#X] (9) InputIteratorTransformer Input [2]: [o_orderkey#X, o_orderpriority#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [l_orderkey#X, l_commitdate#X, l_receiptdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt index 0a9af460526c..da9339abd628 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/5.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (156) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (155) +- Exchange (154) @@ -134,7 +134,7 @@ AdaptiveSparkPlan (156) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -171,7 +171,7 @@ Input [2]: [c_custkey#X, c_nationkey#X] (9) InputIteratorTransformer Input [2]: [c_custkey#X, c_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -240,7 +240,7 @@ Input [2]: [c_nationkey#X, o_orderkey#X] (26) InputIteratorTransformer Input [2]: [c_nationkey#X, o_orderkey#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -309,7 +309,7 @@ Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (43) InputIteratorTransformer Input [4]: [c_nationkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -378,7 +378,7 @@ Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] (60) InputIteratorTransformer Input [3]: [l_extendedprice#X, l_discount#X, s_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [3]: [n_nationkey#X, n_name#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -447,7 +447,7 @@ Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] (77) InputIteratorTransformer Input [4]: [l_extendedprice#X, l_discount#X, n_name#X, n_regionkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt index e9dc68456e55..319c457c1b4f 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/6.txt @@ -10,7 +10,7 @@ AdaptiveSparkPlan (20) +- ^ FlushableHashAggregateExecTransformer (4) +- ^ ProjectExecTransformer (3) +- ^ FilterExecTransformer (2) - +- ^ Scan parquet (1) + +- ^ ScanTransformer parquet (1) +- == Initial Plan == HashAggregate (19) +- Exchange (18) @@ -20,7 +20,7 @@ AdaptiveSparkPlan (20) +- Scan parquet (14) -(1) Scan parquet +(1) ScanTransformer parquet Output [4]: [l_quantity#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt index aa69dbcd11f7..a0894489f0a5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/7.txt @@ -46,35 +46,35 @@ AdaptiveSparkPlan (149) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (81) +- ShuffleQueryStage (79), Statistics(X) +- ReusedExchange (78) @@ -128,7 +128,7 @@ AdaptiveSparkPlan (149) +- Scan parquet (138) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -165,7 +165,7 @@ Input [2]: [s_suppkey#X, s_nationkey#X] (9) InputIteratorTransformer Input [2]: [s_suppkey#X, s_nationkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -234,7 +234,7 @@ Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_ship (26) InputIteratorTransformer Input [5]: [s_nationkey#X, l_orderkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [o_orderkey#X, o_custkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -303,7 +303,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_cust (43) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, o_custkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -372,7 +372,7 @@ Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nati (60) InputIteratorTransformer Input [5]: [s_nationkey#X, l_extendedprice#X, l_discount#X, l_shipdate#X, c_nationkey#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt index 05f9c2c90e5d..8ce3620f30d5 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/8.txt @@ -59,56 +59,56 @@ AdaptiveSparkPlan (207) : : : : : : : +- VeloxResizeBatches (5) : : : : : : : +- ^ ProjectExecTransformer (3) : : : : : : : +- ^ FilterExecTransformer (2) - : : : : : : : +- ^ Scan parquet (1) + : : : : : : : +- ^ ScanTransformer parquet (1) : : : : : : +- ^ InputIteratorTransformer (18) : : : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : : : +- ColumnarExchange (15) : : : : : : +- VeloxResizeBatches (14) : : : : : : +- ^ ProjectExecTransformer (12) : : : : : : +- ^ FilterExecTransformer (11) - : : : : : : +- ^ Scan parquet (10) + : : : : : : +- ^ ScanTransformer parquet (10) : : : : : +- ^ InputIteratorTransformer (35) : : : : : +- ShuffleQueryStage (33), Statistics(X) : : : : : +- ColumnarExchange (32) : : : : : +- VeloxResizeBatches (31) : : : : : +- ^ ProjectExecTransformer (29) : : : : : +- ^ FilterExecTransformer (28) - : : : : : +- ^ Scan parquet (27) + : : : : : +- ^ ScanTransformer parquet (27) : : : : +- ^ InputIteratorTransformer (52) : : : : +- ShuffleQueryStage (50), Statistics(X) : : : : +- ColumnarExchange (49) : : : : +- VeloxResizeBatches (48) : : : : +- ^ ProjectExecTransformer (46) : : : : +- ^ FilterExecTransformer (45) - : : : : +- ^ Scan parquet (44) + : : : : +- ^ ScanTransformer parquet (44) : : : +- ^ InputIteratorTransformer (69) : : : +- ShuffleQueryStage (67), Statistics(X) : : : +- ColumnarExchange (66) : : : +- VeloxResizeBatches (65) : : : +- ^ ProjectExecTransformer (63) : : : +- ^ FilterExecTransformer (62) - : : : +- ^ Scan parquet (61) + : : : +- ^ ScanTransformer parquet (61) : : +- ^ InputIteratorTransformer (86) : : +- ShuffleQueryStage (84), Statistics(X) : : +- ColumnarExchange (83) : : +- VeloxResizeBatches (82) : : +- ^ ProjectExecTransformer (80) : : +- ^ FilterExecTransformer (79) - : : +- ^ Scan parquet (78) + : : +- ^ ScanTransformer parquet (78) : +- ^ InputIteratorTransformer (103) : +- ShuffleQueryStage (101), Statistics(X) : +- ColumnarExchange (100) : +- VeloxResizeBatches (99) : +- ^ ProjectExecTransformer (97) : +- ^ FilterExecTransformer (96) - : +- ^ Scan parquet (95) + : +- ^ ScanTransformer parquet (95) +- ^ InputIteratorTransformer (120) +- ShuffleQueryStage (118), Statistics(X) +- ColumnarExchange (117) +- VeloxResizeBatches (116) +- ^ ProjectExecTransformer (114) +- ^ FilterExecTransformer (113) - +- ^ Scan parquet (112) + +- ^ ScanTransformer parquet (112) +- == Initial Plan == Sort (206) +- Exchange (205) @@ -177,7 +177,7 @@ AdaptiveSparkPlan (207) +- Scan parquet (195) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_type#X] Batched: true Location: InMemoryFileIndex [*] @@ -214,7 +214,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [5]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -283,7 +283,7 @@ Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] (26) InputIteratorTransformer Input [4]: [l_orderkey#X, l_suppkey#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -352,7 +352,7 @@ Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] (43) InputIteratorTransformer Input [4]: [l_orderkey#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [o_orderkey#X, o_custkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -421,7 +421,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_order (60) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_custkey#X, o_orderdate#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [c_custkey#X, c_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -490,7 +490,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nat (77) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, c_nationkey#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_regionkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -559,7 +559,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_reg (94) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, s_nationkey#X, o_orderdate#X, n_regionkey#X] -(95) Scan parquet +(95) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -628,7 +628,7 @@ Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_nam (111) InputIteratorTransformer Input [5]: [l_extendedprice#X, l_discount#X, o_orderdate#X, n_regionkey#X, n_name#X] -(112) Scan parquet +(112) ScanTransformer parquet Output [2]: [r_regionkey#X, r_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt index e298d2a75196..4f537e91e132 100644 --- a/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt +++ b/backends-velox/src/test/resources/tpch-approved-plan/v1/spark34/9.txt @@ -46,42 +46,42 @@ AdaptiveSparkPlan (155) : : : : : +- VeloxResizeBatches (5) : : : : : +- ^ ProjectExecTransformer (3) : : : : : +- ^ FilterExecTransformer (2) - : : : : : +- ^ Scan parquet (1) + : : : : : +- ^ ScanTransformer parquet (1) : : : : +- ^ InputIteratorTransformer (18) : : : : +- ShuffleQueryStage (16), Statistics(X) : : : : +- ColumnarExchange (15) : : : : +- VeloxResizeBatches (14) : : : : +- ^ ProjectExecTransformer (12) : : : : +- ^ FilterExecTransformer (11) - : : : : +- ^ Scan parquet (10) + : : : : +- ^ ScanTransformer parquet (10) : : : +- ^ InputIteratorTransformer (35) : : : +- ShuffleQueryStage (33), Statistics(X) : : : +- ColumnarExchange (32) : : : +- VeloxResizeBatches (31) : : : +- ^ ProjectExecTransformer (29) : : : +- ^ FilterExecTransformer (28) - : : : +- ^ Scan parquet (27) + : : : +- ^ ScanTransformer parquet (27) : : +- ^ InputIteratorTransformer (52) : : +- ShuffleQueryStage (50), Statistics(X) : : +- ColumnarExchange (49) : : +- VeloxResizeBatches (48) : : +- ^ ProjectExecTransformer (46) : : +- ^ FilterExecTransformer (45) - : : +- ^ Scan parquet (44) + : : +- ^ ScanTransformer parquet (44) : +- ^ InputIteratorTransformer (69) : +- ShuffleQueryStage (67), Statistics(X) : +- ColumnarExchange (66) : +- VeloxResizeBatches (65) : +- ^ ProjectExecTransformer (63) : +- ^ FilterExecTransformer (62) - : +- ^ Scan parquet (61) + : +- ^ ScanTransformer parquet (61) +- ^ InputIteratorTransformer (86) +- ShuffleQueryStage (84), Statistics(X) +- ColumnarExchange (83) +- VeloxResizeBatches (82) +- ^ ProjectExecTransformer (80) +- ^ FilterExecTransformer (79) - +- ^ Scan parquet (78) + +- ^ ScanTransformer parquet (78) +- == Initial Plan == Sort (154) +- Exchange (153) @@ -133,7 +133,7 @@ AdaptiveSparkPlan (155) +- Scan parquet (144) -(1) Scan parquet +(1) ScanTransformer parquet Output [2]: [p_partkey#X, p_name#X] Batched: true Location: InMemoryFileIndex [*] @@ -170,7 +170,7 @@ Input [1]: [p_partkey#X] (9) InputIteratorTransformer Input [1]: [p_partkey#X] -(10) Scan parquet +(10) ScanTransformer parquet Output [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] Batched: true Location: InMemoryFileIndex [*] @@ -239,7 +239,7 @@ Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (26) InputIteratorTransformer Input [6]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X] -(27) Scan parquet +(27) ScanTransformer parquet Output [2]: [s_suppkey#X, s_nationkey#X] Batched: true Location: InMemoryFileIndex [*] @@ -308,7 +308,7 @@ Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedpric (43) InputIteratorTransformer Input [7]: [l_orderkey#X, l_partkey#X, l_suppkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X] -(44) Scan parquet +(44) ScanTransformer parquet Output [3]: [ps_partkey#X, ps_suppkey#X, ps_supplycost#X] Batched: true Location: InMemoryFileIndex [*] @@ -377,7 +377,7 @@ Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_natio (60) InputIteratorTransformer Input [6]: [l_orderkey#X, l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X] -(61) Scan parquet +(61) ScanTransformer parquet Output [2]: [o_orderkey#X, o_orderdate#X] Batched: true Location: InMemoryFileIndex [*] @@ -446,7 +446,7 @@ Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_sup (77) InputIteratorTransformer Input [6]: [l_quantity#X, l_extendedprice#X, l_discount#X, s_nationkey#X, ps_supplycost#X, o_orderdate#X] -(78) Scan parquet +(78) ScanTransformer parquet Output [2]: [n_nationkey#X, n_name#X] Batched: true Location: InMemoryFileIndex [*] diff --git a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala index 57fbca17447a..1e378d16f14b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/benchmarks/NativeBenchmarkPlanGenerator.scala @@ -19,18 +19,14 @@ package org.apache.gluten.benchmarks import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.{VeloxWholeStageTransformerSuite, WholeStageTransformer} -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.internal.SQLConf import org.apache.commons.io.FileUtils import org.scalatest.Tag import java.io.File -import java.nio.charset.StandardCharsets -import java.nio.file.{Files, Paths} - -import scala.collection.JavaConverters._ object GenerateExample extends Tag("org.apache.gluten.tags.GenerateExample") @@ -50,6 +46,11 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { createTPCHNotNullTables() } + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + } + test("Test plan json non-empty - AQE off") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", @@ -67,7 +68,6 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } - spark.sparkContext.setLogLevel(logLevel) } test("Test plan json non-empty - AQE on") { @@ -88,70 +88,42 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { val planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } - spark.sparkContext.setLogLevel(logLevel) } test("generate example", GenerateExample) { - import testImplicits._ withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", SQLConf.SHUFFLE_PARTITIONS.key -> "2", - GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true" + GlutenConfig.BENCHMARK_SAVE_DIR.key -> generatedPlanDir, + GlutenConfig.BENCHMARK_TASK_STAGEID.key -> "12", + GlutenConfig.BENCHMARK_TASK_PARTITIONID.key -> "0" ) { logWarning(s"Generating inputs for micro benchmark to $generatedPlanDir") - val q4_lineitem = spark - .sql(s""" - |select l_orderkey from lineitem where l_commitdate < l_receiptdate - |""".stripMargin) - val q4_orders = spark - .sql(s""" - |select o_orderkey, o_orderpriority - | from orders - | where o_orderdate >= '1993-07-01' and o_orderdate < '1993-10-01' - |""".stripMargin) - q4_lineitem - .createOrReplaceTempView("q4_lineitem") - q4_orders - .createOrReplaceTempView("q4_orders") - - q4_lineitem - .repartition(1, 'l_orderkey) - .write - .format(outputFileFormat) - .save(generatedPlanDir + "/example_lineitem") - q4_orders - .repartition(1, 'o_orderkey) - .write - .format(outputFileFormat) - .save(generatedPlanDir + "/example_orders") - - val df = - spark.sql(""" - |select * from q4_orders left semi join q4_lineitem on l_orderkey = o_orderkey - |""".stripMargin) - generateSubstraitJson(df, "example.json") + spark + .sql(""" + |select /*+ REPARTITION(1) */ + | o_orderpriority, + | count(*) as order_count + |from + | orders + |where + | o_orderdate >= date '1993-07-01' + | and o_orderdate < date '1993-07-01' + interval '3' month + | and exists ( + | select /*+ REPARTITION(1) */ + | * + | from + | lineitem + | where + | l_orderkey = o_orderkey + | and l_commitdate < l_receiptdate + | ) + |group by + | o_orderpriority + |order by + | o_orderpriority + |""".stripMargin) + .foreach(_ => ()) } - spark.sparkContext.setLogLevel(logLevel) - } - - def generateSubstraitJson(df: DataFrame, file: String): Unit = { - val executedPlan = df.queryExecution.executedPlan - executedPlan.execute() - val finalPlan = - executedPlan match { - case aqe: AdaptiveSparkPlanExec => - aqe.executedPlan match { - case s: ShuffleQueryStageExec => s.shuffle.child - case other => other - } - case plan => plan - } - val lastStageTransformer = finalPlan.find(_.isInstanceOf[WholeStageTransformer]) - assert(lastStageTransformer.nonEmpty) - val plan = - lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson.split('\n') - - val exampleJsonFile = Paths.get(generatedPlanDir, file) - Files.write(exampleJsonFile, plan.toList.asJava, StandardCharsets.UTF_8) } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala index d8b0ee8981e2..0f94b8648e71 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/FallbackSuite.scala @@ -17,7 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.sql.execution.{ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, SparkPlan} @@ -253,7 +252,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl test("fallback with smj") { val sql = "SELECT /*+ SHUFFLE_MERGE(tmp1) */ * FROM tmp1 join tmp2 on tmp1.c1 = tmp2.c1" withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "true", GlutenConfig.COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => @@ -262,7 +261,7 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl } } withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", GlutenConfig.COLUMNAR_SORTMERGEJOIN_ENABLED.key -> "false") { runQueryAndCompare(sql) { df => diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala index 7e9b48e96ca9..8063a5d12207 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/MiscOperatorSuite.scala @@ -208,6 +208,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa "select l_orderkey from lineitem " + "where l_partkey in (1552, 674) or l_partkey in (1552) and l_orderkey > 1") { _ => } checkLengthAndPlan(df, 73) + + runQueryAndCompare( + "select count(1) from lineitem " + + "where (l_shipmode in ('TRUCK', 'MAIL') or l_shipmode in ('AIR', 'FOB')) " + + "and l_shipmode in ('RAIL','SHIP')") { + checkGlutenOperatorMatch[FileSourceScanExecTransformer] + } } test("in_not") { @@ -505,6 +512,13 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa checkGlutenOperatorMatch[WindowExecTransformer] } } + + // Foldable input of nth_value is not supported. + runQueryAndCompare( + "select l_suppkey, l_orderkey, nth_value(1, 2) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { + checkSparkOperatorMatch[WindowExec] + } } } @@ -523,11 +537,37 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa |""".stripMargin) { df => { - getExecutedPlan(df).exists(plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined) + assert( + getExecutedPlan(df).exists( + plan => plan.find(_.isInstanceOf[ColumnarUnionExec]).isDefined)) } } } + test("union_all two tables with known partitioning") { + withSQLConf(GlutenConfig.NATIVE_UNION_ENABLED.key -> "true") { + compareDfResultsAgainstVanillaSpark( + () => { + val df1 = spark.sql("select l_orderkey as orderkey from lineitem") + val df2 = spark.sql("select o_orderkey as orderkey from orders") + df1.repartition(5).union(df2.repartition(5)) + }, + compareResult = true, + checkGlutenOperatorMatch[UnionExecTransformer] + ) + + compareDfResultsAgainstVanillaSpark( + () => { + val df1 = spark.sql("select l_orderkey as orderkey from lineitem") + val df2 = spark.sql("select o_orderkey as orderkey from orders") + df1.repartition(5).union(df2.repartition(6)) + }, + compareResult = true, + checkGlutenOperatorMatch[ColumnarUnionExec] + ) + } + } + test("union_all three tables") { runQueryAndCompare(""" |select count(orderkey) from ( @@ -566,7 +606,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa | select * from lineitem limit 10 |) where l_suppkey != 0 limit 100; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } } @@ -845,7 +885,7 @@ class MiscOperatorSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1" ) { val query = diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala index 84d1fe8db919..cf0bf272bc5b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxAggregateFunctionsSuite.scala @@ -49,6 +49,7 @@ abstract class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSu .set("spark.unsafe.exceptionOnMemoryLeak", "true") .set("spark.sql.autoBroadcastJoinThreshold", "-1") .set("spark.sql.sources.useV1SourceList", "avro") + .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "false") } test("count") { diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala index 4a251449845f..e41f00821c27 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxMetricsSuite.scala @@ -22,8 +22,9 @@ import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted} import org.apache.spark.sql.TestUtils -import org.apache.spark.sql.execution.CommandResultExec -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.{ColumnarInputAdapter, CommandResultExec, InputIteratorTransformer} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, BroadcastQueryStageExec} +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.internal.SQLConf class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper { @@ -62,7 +63,7 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa test("test sort merge join metrics") { withSQLConf( - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { // without preproject runQueryAndCompare( @@ -227,4 +228,39 @@ class VeloxMetricsSuite extends VeloxWholeStageTransformerSuite with AdaptiveSpa assert(inputRecords == (partTableRecords + itemTableRecords)) } + + test("Metrics for input iterator of broadcast exchange") { + createTPCHNotNullTables() + val partTableRecords = spark.sql("select * from part").count() + + // Repartition to make sure we have multiple tasks executing the join. + spark + .sql("select * from lineitem") + .repartition(2) + .createOrReplaceTempView("lineitem") + + Seq("true", "false").foreach { + adaptiveEnabled => + withSQLConf("spark.sql.adaptive.enabled" -> adaptiveEnabled) { + val sqlStr = + """ + |select /*+ BROADCAST(part) */ * from part join lineitem + |on l_partkey = p_partkey + |""".stripMargin + + runQueryAndCompare(sqlStr) { + df => + val inputIterator = find(df.queryExecution.executedPlan) { + case InputIteratorTransformer(ColumnarInputAdapter(child)) => + child.isInstanceOf[BroadcastQueryStageExec] || child + .isInstanceOf[BroadcastExchangeLike] + case _ => false + } + assert(inputIterator.isDefined) + val metrics = inputIterator.get.metrics + assert(metrics("numOutputRows").value == partTableRecords) + } + } + } + } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala index 3bf49d33c511..6ac59ba4fa6b 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxOrcDataTypeValidationSuite.scala @@ -255,7 +255,10 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { |""".stripMargin) { df => { - assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec])) + assert( + getExecutedPlan(df).exists( + plan => + plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer])) } } @@ -265,7 +268,7 @@ class VeloxOrcDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { | select date, int from type1 limit 100 |) where int != 0 limit 10; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } // Validation: Window. diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala index 8b6cc63c954d..cb5614f39669 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxParquetDataTypeValidationSuite.scala @@ -254,7 +254,10 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit |""".stripMargin) { df => { - assert(getExecutedPlan(df).exists(plan => plan.isInstanceOf[ColumnarUnionExec])) + assert( + getExecutedPlan(df).exists( + plan => + plan.isInstanceOf[ColumnarUnionExec] || plan.isInstanceOf[UnionExecTransformer])) } } @@ -264,7 +267,7 @@ class VeloxParquetDataTypeValidationSuite extends VeloxWholeStageTransformerSuit | select date, int from type1 limit 100 |) where int != 0 limit 10; |""".stripMargin) { - checkGlutenOperatorMatch[LimitTransformer] + checkGlutenOperatorMatch[LimitExecTransformer] } // Validation: Window. diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala index a50bffa3edcd..a4f16ecc3c8f 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxScanSuite.scala @@ -18,11 +18,13 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.velox.VeloxBackendSettings +import org.apache.gluten.benchmarks.RandomParquetDataGenerator import org.apache.gluten.utils.VeloxFileSystemValidationJniWrapper import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.GreaterThan import org.apache.spark.sql.execution.ScalarSubquery +import org.apache.spark.sql.types._ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath @@ -114,4 +116,38 @@ class VeloxScanSuite extends VeloxWholeStageTransformerSuite { !VeloxFileSystemValidationJniWrapper.allSupportedByRegisteredFileSystems( Array("file:/test_path/", "unsupported://test_path"))) } + + test("scan with filter on decimal/timestamp/binary field") { + withTempView("t") { + withTempDir { + dir => + val path = dir.getAbsolutePath + val schema = StructType( + Array( + StructField("short_decimal_field", DecimalType(5, 2), nullable = true), + StructField("long_decimal_field", DecimalType(32, 8), nullable = true), + StructField("binary_field", BinaryType, nullable = true), + StructField("timestamp_field", TimestampType, nullable = true) + )) + RandomParquetDataGenerator(0).generateRandomData(spark, schema, 10, Some(path)) + spark.catalog.createTable("t", path, "parquet") + + runQueryAndCompare( + """select * from t where long_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where short_decimal_field = 3.14""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where binary_field = '3.14'""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + + runQueryAndCompare( + """select * from t where timestamp_field = current_timestamp()""".stripMargin + )(checkGlutenOperatorMatch[FileSourceScanExecTransformer]) + } + } + } } diff --git a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala index 1bef5f991fac..f1e5a42bc0a3 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/execution/VeloxStringFunctionsSuite.scala @@ -268,7 +268,7 @@ class VeloxStringFunctionsSuite extends VeloxWholeStageTransformerSuite { s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer]) } - ignore("locate") { + test("locate") { runQueryAndCompare( s"select l_orderkey, locate(l_comment, 'a', 1) " + s"from $LINEITEM_TABLE limit $LENGTH")(checkGlutenOperatorMatch[ProjectExecTransformer]) diff --git a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala index f85103deb847..61ba927cd457 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/expression/VeloxUdfSuite.scala @@ -16,11 +16,13 @@ */ package org.apache.gluten.expression +import org.apache.gluten.execution.ProjectExecTransformer import org.apache.gluten.tags.{SkipTestTags, UDFTest} import org.apache.spark.SparkConf import org.apache.spark.sql.{GlutenQueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.expression.UDFResolver import java.nio.file.Paths @@ -158,16 +160,24 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper { |AS 'org.apache.spark.sql.hive.execution.UDFStringString' |""".stripMargin) - val nativeResultWithImplicitConversion = - spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""").collect() - val nativeResult = - spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect() + val offloadWithImplicitConversionDF = + spark.sql(s"""SELECT hive_string_string(col1, 'a') FROM $tbl""") + checkGlutenOperatorMatch[ProjectExecTransformer](offloadWithImplicitConversionDF) + val offloadWithImplicitConversionResult = offloadWithImplicitConversionDF.collect() + + val offloadDF = + spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""") + checkGlutenOperatorMatch[ProjectExecTransformer](offloadDF) + val offloadResult = offloadWithImplicitConversionDF.collect() + // Unregister native hive udf to fallback. UDFResolver.UDFNames.remove("org.apache.spark.sql.hive.execution.UDFStringString") - val fallbackResult = - spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""").collect() - assert(nativeResultWithImplicitConversion.sameElements(fallbackResult)) - assert(nativeResult.sameElements(fallbackResult)) + val fallbackDF = + spark.sql(s"""SELECT hive_string_string(col2, 'a') FROM $tbl""") + checkSparkOperatorMatch[ProjectExec](fallbackDF) + val fallbackResult = fallbackDF.collect() + assert(offloadWithImplicitConversionResult.sameElements(fallbackResult)) + assert(offloadResult.sameElements(fallbackResult)) // Add an unimplemented udf to the map to test fallback of registered native hive udf. UDFResolver.UDFNames.add("org.apache.spark.sql.hive.execution.UDFIntegerToString") @@ -176,6 +186,7 @@ abstract class VeloxUdfSuite extends GlutenQueryTest with SQLHelper { |AS 'org.apache.spark.sql.hive.execution.UDFIntegerToString' |""".stripMargin) val df = spark.sql(s"""select hive_int_to_string(col1) from $tbl""") + checkSparkOperatorMatch[ProjectExec](df) checkAnswer(df, Seq(Row("1"), Row("2"), Row("3"))) } finally { spark.sql(s"DROP TABLE IF EXISTS $tbl") diff --git a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala index d12faae0f73d..e14ffd43d82d 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/extension/columnar/transition/VeloxTransitionSuite.scala @@ -33,37 +33,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla C2R - outputs row") { val in = BatchLeaf(VanillaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(VanillaBatch))) } test("Vanilla C2R - requires row input") { val in = RowUnary(BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(VanillaBatch)))) } test("Vanilla R2C - requires vanilla input") { val in = BatchUnary(VanillaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, RowToColumnarExec(RowLeaf())))) } test("ArrowNative C2R - outputs row") { val in = BatchLeaf(ArrowNativeBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch)))) } test("ArrowNative C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) } test("ArrowNative R2C - requires Arrow input") { val in = BatchUnary(ArrowNativeBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, RowToVeloxColumnarExec(RowLeaf()))))) @@ -71,7 +71,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) // No explicit transition needed for ArrowNative-to-Velox. // FIXME: Add explicit transitions. // See https://github.com/apache/incubator-gluten/issues/7313. @@ -82,7 +82,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary(ArrowNativeBatch, BatchLeaf(VeloxBatch))))) @@ -90,7 +90,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowNative C2C") { val in = BatchUnary(ArrowNativeBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( LoadArrowDataExec(BatchUnary( @@ -100,7 +100,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowNative-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowNativeBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(ArrowNativeBatch))))) @@ -108,19 +108,19 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava C2R - outputs row") { val in = BatchLeaf(ArrowJavaBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchLeaf(ArrowJavaBatch))) } test("ArrowJava C2R - requires row input") { val in = RowUnary(BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(ColumnarToRowExec(BatchLeaf(ArrowJavaBatch)))) } test("ArrowJava R2C - requires Arrow input") { val in = BatchUnary(ArrowJavaBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(RowToVeloxColumnarExec(RowLeaf()))))) @@ -128,7 +128,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary( @@ -138,7 +138,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary(ArrowJavaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) @@ -146,7 +146,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Vanilla-to-ArrowJava C2C") { val in = BatchUnary(ArrowJavaBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec( BatchUnary( @@ -156,37 +156,37 @@ class VeloxTransitionSuite extends SharedSparkSession { test("ArrowJava-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == ColumnarToRowExec(BatchUnary(VanillaBatch, BatchLeaf(ArrowJavaBatch)))) } test("Velox C2R - outputs row") { val in = BatchLeaf(VeloxBatch) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchLeaf(VeloxBatch))) } test("Velox C2R - requires row input") { val in = RowUnary(BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == RowUnary(VeloxColumnarToRowExec(BatchLeaf(VeloxBatch)))) } test("Velox R2C - outputs Velox") { val in = RowLeaf() - val out = Transitions.insertTransitions(in, outputsColumnar = true) + val out = BackendTransitions.insert(in, outputsColumnar = true) assert(out == RowToVeloxColumnarExec(RowLeaf())) } test("Velox R2C - requires Velox input") { val in = BatchUnary(VeloxBatch, RowLeaf()) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert(out == VeloxColumnarToRowExec(BatchUnary(VeloxBatch, RowToVeloxColumnarExec(RowLeaf())))) } test("Vanilla-to-Velox C2C") { val in = BatchUnary(VeloxBatch, BatchLeaf(VanillaBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == VeloxColumnarToRowExec( BatchUnary(VeloxBatch, RowToVeloxColumnarExec(ColumnarToRowExec(BatchLeaf(VanillaBatch)))))) @@ -194,7 +194,7 @@ class VeloxTransitionSuite extends SharedSparkSession { test("Velox-to-Vanilla C2C") { val in = BatchUnary(VanillaBatch, BatchLeaf(VeloxBatch)) - val out = Transitions.insertTransitions(in, outputsColumnar = false) + val out = BackendTransitions.insert(in, outputsColumnar = false) assert( out == ColumnarToRowExec(BatchUnary(VanillaBatch, LoadArrowDataExec(BatchLeaf(VeloxBatch))))) } diff --git a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala index 38a6832c4e3f..885b549d0def 100644 --- a/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala +++ b/backends-velox/src/test/scala/org/apache/gluten/planner/VeloxRasSuite.scala @@ -17,9 +17,11 @@ package org.apache.gluten.planner import org.apache.gluten.GlutenConfig +import org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform +import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LegacyCoster, LongCostModel} +import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.planner.cost.GlutenCostModel -import org.apache.gluten.planner.property.Conv import org.apache.gluten.ras.{Cost, CostModel, Ras} import org.apache.gluten.ras.RasSuiteBase._ import org.apache.gluten.ras.path.RasPath @@ -140,23 +142,30 @@ class VeloxRasSuite extends SharedSparkSession { object VeloxRasSuite { def newRas(): Ras[SparkPlan] = { - GlutenOptimization - .builder() - .costModel(GlutenCostModel.find()) - .addRules(List()) - .create() - .asInstanceOf[Ras[SparkPlan]] + newRas(Nil) } - def newRas(RasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = { + def newRas(rasRules: Seq[RasRule[SparkPlan]]): Ras[SparkPlan] = { GlutenOptimization .builder() - .costModel(GlutenCostModel.find()) - .addRules(RasRules) + .costModel(sessionCostModel()) + .addRules(rasRules) .create() .asInstanceOf[Ras[SparkPlan]] } + private def legacyCostModel(): CostModel[SparkPlan] = { + val registry = LongCostModel.registry() + val coster = LegacyCoster + registry.register(coster) + registry.get(coster.kind()) + } + + private def sessionCostModel(): CostModel[SparkPlan] = { + val transform = EnumeratedTransform.static() + transform.costModel + } + val TRIVIAL_SCHEMA: Seq[AttributeReference] = List(AttributeReference("value", StringType)()) val EMPTY_SCHEMA: Seq[AttributeReference] = List.empty @@ -191,7 +200,7 @@ object VeloxRasSuite { } class UserCostModel1 extends CostModel[SparkPlan] { - private val base = GlutenCostModel.legacy() + private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: RowUnary => base.makeInfCost() case other => base.costOf(other) @@ -201,7 +210,7 @@ object VeloxRasSuite { } class UserCostModel2 extends CostModel[SparkPlan] { - private val base = GlutenCostModel.legacy() + private val base = legacyCostModel() override def costOf(node: SparkPlan): Cost = node match { case _: ColumnarUnary => base.makeInfCost() case other => base.costOf(other) diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala index 412548de9c44..5932f4e5a741 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteForHiveSuite.scala @@ -27,6 +27,11 @@ import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.util.QueryExecutionListener +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.util.HadoopInputFile class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { private var _spark: SparkSession = _ @@ -176,4 +181,45 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { checkAnswer(sql("SELECT * FROM t"), Row(1)) } } + + test("native writer support CreateHiveTableAsSelectCommand") { + withTable("t") { + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + checkNativeWrite("CREATE TABLE t STORED AS PARQUET AS SELECT 1 as c", checkNative = true) + } + checkAnswer(spark.table("t"), Row(1)) + } + } + + test("native writer should respect table properties") { + Seq(true, false).foreach { + enableNativeWrite => + withSQLConf("spark.gluten.sql.native.writer.enabled" -> enableNativeWrite.toString) { + withTable("t") { + withSQLConf( + "spark.sql.hive.convertMetastoreParquet" -> "false", + "spark.sql.parquet.compression.codec" -> "gzip") { + checkNativeWrite( + "CREATE TABLE t STORED AS PARQUET TBLPROPERTIES ('parquet.compression'='zstd') " + + "AS SELECT 1 as c", + checkNative = enableNativeWrite) + val tableDir = new Path(s"${conf.getConf(StaticSQLConf.WAREHOUSE_PATH)}/t") + val configuration = spark.sessionState.newHadoopConf() + val files = tableDir + .getFileSystem(configuration) + .listStatus(tableDir) + .filterNot(_.getPath.getName.startsWith("\\.")) + assert(files.nonEmpty) + val in = HadoopInputFile.fromStatus(files.head, spark.sessionState.newHadoopConf()) + Utils.tryWithResource(ParquetFileReader.open(in)) { + reader => + val column = reader.getFooter.getBlocks.get(0).getColumns.get(0) + // native writer and vanilla spark hive writer should be consistent + "zstd".equalsIgnoreCase(column.getCodec.toString) + } + } + } + } + } + } } diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala index 4c76c753b90e..d19d279fbb2c 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/VeloxParquetWriteSuite.scala @@ -31,6 +31,22 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { override protected val resourcePath: String = "/tpch-data-parquet" override protected val fileFormat: String = "parquet" + // The parquet compression codec extensions + private val parquetCompressionCodecExtensions = Map( + "none" -> "", + "uncompressed" -> "", + "snappy" -> ".snappy", + "gzip" -> ".gz", + "lzo" -> ".lzo", + "lz4" -> ".lz4", + "brotli" -> ".br", + "zstd" -> ".zstd" + ) + + private def getParquetFileExtension(codec: String): String = { + s"${parquetCompressionCodecExtensions(codec)}.parquet" + } + override def beforeAll(): Unit = { super.beforeAll() createTPCHNotNullTables() @@ -49,8 +65,8 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { spark.sql("select array(struct(1), null) as var1").write.mode("overwrite").save(path) } assert( - testAppender.loggingEvents.exists( - _.getMessage.toString.contains("Use Gluten parquet write for hive")) == false) + !testAppender.loggingEvents.exists( + _.getMessage.toString.contains("Use Gluten parquet write for hive"))) } } @@ -77,6 +93,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { parquetFiles.forall { file => val path = new Path(f.getCanonicalPath, file) + assert(file.endsWith(getParquetFileExtension(codec))) val in = HadoopInputFile.fromPath(path, spark.sessionState.newHadoopConf()) Utils.tryWithResource(ParquetFileReader.open(in)) { reader => diff --git a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala index 7f11fd043db4..77370d059caa 100644 --- a/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala +++ b/backends-velox/src/test/scala/org/apache/spark/sql/execution/benchmark/StreamingAggregateBenchmark.scala @@ -51,7 +51,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "true", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } @@ -62,7 +62,7 @@ object StreamingAggregateBenchmark extends SqlBasedBenchmark { withSQLConf( GlutenConfig.COLUMNAR_PREFER_STREAMING_AGGREGATE.key -> "false", SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" + GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key -> "false" ) { spark.sql(query).noop() } diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 445cd99068a1..7c94e253c7b7 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20241101 -CH_COMMIT=7cd7bb8ece2 \ No newline at end of file +CH_BRANCH=rebase_ch/20241205 +CH_COMMIT=7b8b1c0b554 diff --git a/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp new file mode 100644 index 000000000000..67c9e8380aa3 --- /dev/null +++ b/cpp-ch/local-engine/AggregateFunctions/GroupLimitFunctions.cpp @@ -0,0 +1,296 @@ +/* + * 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. + */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int LOGICAL_ERROR; +extern const int BAD_ARGUMENTS; +} + +namespace local_engine +{ + +struct SortOrderField +{ + size_t pos = 0; + Int8 direction = 0; + Int8 nulls_direction = 0; +}; +using SortOrderFields = std::vector; + +struct RowNumGroupArraySortedData +{ +public: + using Data = DB::Tuple; + std::vector values; + + static bool compare(const Data & lhs, const Data & rhs, const SortOrderFields & sort_orders) + { + for (const auto & sort_order : sort_orders) + { + const auto & pos = sort_order.pos; + const auto & asc = sort_order.direction; + const auto & nulls_first = sort_order.nulls_direction; + bool l_is_null = lhs[pos].isNull(); + bool r_is_null = rhs[pos].isNull(); + if (l_is_null && r_is_null) + continue; + else if (l_is_null) + return nulls_first; + else if (r_is_null) + return !nulls_first; + else if (lhs[pos] < rhs[pos]) + return asc; + else if (lhs[pos] > rhs[pos]) + return !asc; + } + return false; + } + + ALWAYS_INLINE void heapReplaceTop(const SortOrderFields & sort_orders) + { + size_t size = values.size(); + if (size < 2) + return; + size_t child_index = 1; + if (size > 2 && compare(values[1], values[2], sort_orders)) + ++child_index; + + if (compare(values[child_index], values[0], sort_orders)) + return; + + size_t current_index = 0; + auto current = values[current_index]; + do + { + values[current_index] = values[child_index]; + current_index = child_index; + + child_index = 2 * child_index + 1; + + if (child_index >= size) + break; + + if ((child_index + 1) < size && compare(values[child_index], values[child_index + 1], sort_orders)) + ++child_index; + } while (!compare(values[child_index], current, sort_orders)); + + values[current_index] = current; + } + + ALWAYS_INLINE void addElement(const Data && data, const SortOrderFields & sort_orders, size_t max_elements) + { + if (values.size() >= max_elements) + { + if (!compare(data, values[0], sort_orders)) + return; + values[0] = data; + heapReplaceTop(sort_orders); + return; + } + values.emplace_back(std::move(data)); + auto cmp = [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); }; + std::push_heap(values.begin(), values.end(), cmp); + } + + ALWAYS_INLINE void sortAndLimit(size_t max_elements, const SortOrderFields & sort_orders) + { + ::sort(values.begin(), values.end(), [&sort_orders](const Data & a, const Data & b) { return compare(a, b, sort_orders); }); + if (values.size() > max_elements) + values.resize(max_elements); + } + + ALWAYS_INLINE void insertResultInto(DB::IColumn & to, size_t max_elements, const SortOrderFields & sort_orders) + { + auto & result_array = assert_cast(to); + auto & result_array_offsets = result_array.getOffsets(); + + sortAndLimit(max_elements, sort_orders); + + result_array_offsets.push_back(result_array_offsets.back() + values.size()); + + if (values.empty()) + return; + auto & result_array_data = result_array.getData(); + for (int i = 0, sz = static_cast(values.size()); i < sz; ++i) + { + auto & value = values[i]; + value.push_back(i + 1); + result_array_data.insert(value); + } + } +}; + +static DB::DataTypePtr getRowNumReultDataType(DB::DataTypePtr data_type) +{ + const auto * tuple_type = typeid_cast(data_type.get()); + if (!tuple_type) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName()); + DB::DataTypes element_types = tuple_type->getElements(); + std::vector element_names = tuple_type->getElementNames(); + element_types.push_back(std::make_shared()); + element_names.push_back("row_num"); + auto nested_tuple_type = std::make_shared(element_types, element_names); + return std::make_shared(nested_tuple_type); +} + +// usage: rowNumGroupArraySorted(1, "a asc nulls first, b desc nulls last")(tuple(a,b)) +class RowNumGroupArraySorted final : public DB::IAggregateFunctionDataHelper +{ +public: + explicit RowNumGroupArraySorted(DB::DataTypePtr data_type, const DB::Array & parameters_) + : DB::IAggregateFunctionDataHelper( + {data_type}, parameters_, getRowNumReultDataType(data_type)) + { + if (parameters_.size() != 2) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "{} needs two parameters: limit and order clause", getName()); + const auto * tuple_type = typeid_cast(data_type.get()); + if (!tuple_type) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Tuple type is expected, but got: {}", data_type->getName()); + + limit = parameters_[0].safeGet(); + + String order_by_clause = parameters_[1].safeGet(); + sort_order_fields = parseSortOrderFields(order_by_clause); + + serialization = data_type->getDefaultSerialization(); + } + + String getName() const override { return "rowNumGroupArraySorted"; } + + void add(DB::AggregateDataPtr __restrict place, const DB::IColumn ** columns, size_t row_num, DB::Arena * /*arena*/) const override + { + auto & data = this->data(place); + DB::Tuple data_tuple = (*columns[0])[row_num].safeGet(); + this->data(place).addElement(std::move(data_tuple), sort_order_fields, limit); + } + + void merge(DB::AggregateDataPtr __restrict place, DB::ConstAggregateDataPtr rhs, DB::Arena * /*arena*/) const override + { + auto & rhs_values = this->data(rhs).values; + for (auto & rhs_element : rhs_values) + this->data(place).addElement(std::move(rhs_element), sort_order_fields, limit); + } + + void serialize(DB::ConstAggregateDataPtr __restrict place, DB::WriteBuffer & buf, std::optional /* version */) const override + { + auto & values = this->data(place).values; + size_t size = values.size(); + DB::writeVarUInt(size, buf); + + for (const auto & value : values) + serialization->serializeBinary(value, buf, {}); + } + + void deserialize( + DB::AggregateDataPtr __restrict place, DB::ReadBuffer & buf, std::optional /* version */, DB::Arena *) const override + { + size_t size = 0; + DB::readVarUInt(size, buf); + + auto & values = this->data(place).values; + values.reserve(size); + for (size_t i = 0; i < size; ++i) + { + DB::Field data; + serialization->deserializeBinary(data, buf, {}); + values.emplace_back(data.safeGet()); + } + } + + void insertResultInto(DB::AggregateDataPtr __restrict place, DB::IColumn & to, DB::Arena * /*arena*/) const override + { + this->data(place).insertResultInto(to, limit, sort_order_fields); + } + + bool allocatesMemoryInArena() const override { return true; } + +private: + size_t limit = 0; + SortOrderFields sort_order_fields; + DB::SerializationPtr serialization; + + SortOrderFields parseSortOrderFields(const String & order_by_clause) const + { + DB::ParserOrderByExpressionList order_by_parser; + auto order_by_ast = DB::parseQuery(order_by_parser, order_by_clause, 1000, 1000, 1000); + SortOrderFields fields; + const auto expression_list_ast = assert_cast(order_by_ast.get()); + const auto & tuple_element_names = assert_cast(argument_types[0].get())->getElementNames(); + for (const auto & child : expression_list_ast->children) + { + const auto * order_by_element_ast = assert_cast(child.get()); + const auto * ident_ast = assert_cast(order_by_element_ast->children[0].get()); + const auto & ident_name = ident_ast->shortName(); + + + SortOrderField field; + field.direction = order_by_element_ast->direction == 1; + field.nulls_direction + = field.direction ? order_by_element_ast->nulls_direction == -1 : order_by_element_ast->nulls_direction == 1; + + auto name_pos = std::find(tuple_element_names.begin(), tuple_element_names.end(), ident_name); + if (name_pos == tuple_element_names.end()) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Not found column {} in tuple {}", ident_name, argument_types[0]->getName()); + } + field.pos = std::distance(tuple_element_names.begin(), name_pos); + + fields.push_back(field); + } + return fields; + } +}; + + +DB::AggregateFunctionPtr createAggregateFunctionRowNumGroupArray( + const std::string & name, const DB::DataTypes & argument_types, const DB::Array & parameters, const DB::Settings *) +{ + if (argument_types.size() != 1 || !typeid_cast(argument_types[0].get())) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, " {} Nees only one tuple argument", name); + return std::make_shared(argument_types[0], parameters); +} + +void registerAggregateFunctionRowNumGroup(DB::AggregateFunctionFactory & factory) +{ + DB::AggregateFunctionProperties properties = {.returns_default_when_only_null = false, .is_order_dependent = false}; + + factory.registerFunction("rowNumGroupArraySorted", {createAggregateFunctionRowNumGroupArray, properties}); +} +} diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index 4392b55008bf..4b1c64363646 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -174,6 +174,26 @@ else() set(LOCALENGINE_SHARED_LIB_NAME "libch.so") endif() +option(ENABLE_SEPARATE_SYMBOLS "support separate debug symbols from so" OFF) +if(ENABLE_SEPARATE_SYMBOLS) + set(SYMBOL_OUTPUT_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/debug_symbols) + file(MAKE_DIRECTORY ${SYMBOL_OUTPUT_DIRECTORY}) + function(separate_symbols target) + add_custom_command( + TARGET ${target} + POST_BUILD + COMMAND ${CMAKE_OBJCOPY} --only-keep-debug $ + ${SYMBOL_OUTPUT_DIRECTORY}/$.debug + COMMAND ${CMAKE_OBJCOPY} --strip-debug $ + COMMAND + ${CMAKE_OBJCOPY} + --add-gnu-debuglink=${SYMBOL_OUTPUT_DIRECTORY}/$.debug + $ + COMMENT "Separating debug symbols for target: ${target}") + endfunction() + separate_symbols(${LOCALENGINE_SHARED_LIB}) +endif() + add_custom_command( OUTPUT ${LOCALENGINE_SHARED_LIB_NAME} COMMAND ${CMAKE_COMMAND} -E rename $ diff --git a/cpp-ch/local-engine/Common/AggregateUtil.cpp b/cpp-ch/local-engine/Common/AggregateUtil.cpp index 851dd2e7fe3d..36b26e94bd17 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.cpp +++ b/cpp-ch/local-engine/Common/AggregateUtil.cpp @@ -15,8 +15,11 @@ * limitations under the License. */ +#include "AggregateUtil.h" +#include #include #include +#include #include #include #include @@ -26,8 +29,29 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; - extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; +extern const int LOGICAL_ERROR; +extern const int UNKNOWN_AGGREGATED_DATA_VARIANT; +} + +namespace Setting +{ +extern const SettingsDouble max_bytes_ratio_before_external_group_by; +extern const SettingsUInt64 max_bytes_before_external_group_by; +extern const SettingsBool optimize_group_by_constant_keys; +extern const SettingsUInt64 min_free_disk_space_for_temporary_data; +extern const SettingsMaxThreads max_threads; +extern const SettingsBool empty_result_for_aggregation_by_empty_set; +extern const SettingsUInt64 group_by_two_level_threshold_bytes; +extern const SettingsOverflowModeGroupBy group_by_overflow_mode; +extern const SettingsUInt64 max_rows_to_group_by; +extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; +extern const SettingsUInt64 aggregation_in_order_max_block_bytes; +extern const SettingsUInt64 group_by_two_level_threshold; +extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; +extern const SettingsUInt64 max_block_size; +extern const SettingsBool compile_aggregate_expressions; +extern const SettingsUInt64 min_count_to_compile_aggregate_expression; +extern const SettingsBool enable_software_prefetch_in_aggregation; } template @@ -39,24 +63,23 @@ static Int32 extractMethodBucketsNum(Method & /*method*/) Int32 GlutenAggregatorUtil::getBucketsNum(AggregatedDataVariants & data_variants) { if (!data_variants.isTwoLevel()) - { return 0; - } - + Int32 buckets_num = 0; #define M(NAME) \ - else if (data_variants.type == AggregatedDataVariants::Type::NAME) \ - buckets_num = extractMethodBucketsNum(*data_variants.NAME); + else if (data_variants.type == AggregatedDataVariants::Type::NAME) buckets_num = extractMethodBucketsNum(*data_variants.NAME); - if (false) {} // NOLINT + if (false) + { + } // NOLINT APPLY_FOR_VARIANTS_TWO_LEVEL(M) #undef M - else - throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); + else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); return buckets_num; } -std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket) +std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock( + Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket) { if (!variants.isTwoLevel()) return {}; @@ -65,7 +88,7 @@ std::optional GlutenAggregatorUtil::safeConvertOneBucketToBlock(Aggregato return aggregator.convertOneBucketToBlock(variants, arena, final, bucket); } -template +template static void releaseOneBucket(Method & method, Int32 bucket) { method.data.impls[bucket].clearAndShrink(); @@ -77,29 +100,26 @@ void GlutenAggregatorUtil::safeReleaseOneBucket(AggregatedDataVariants & variant return; if (bucket >= getBucketsNum(variants)) return; -#define M(NAME) \ - else if (variants.type == AggregatedDataVariants::Type::NAME) \ - releaseOneBucket(*variants.NAME, bucket); +#define M(NAME) else if (variants.type == AggregatedDataVariants::Type::NAME) releaseOneBucket(*variants.NAME, bucket); - if (false) {} // NOLINT + if (false) + { + } // NOLINT APPLY_FOR_VARIANTS_TWO_LEVEL(M) #undef M - else - throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); - + else throw Exception(ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT, "Unknown aggregated data variant"); } } namespace local_engine { -AggregateDataBlockConverter::AggregateDataBlockConverter(DB::Aggregator & aggregator_, DB::AggregatedDataVariantsPtr data_variants_, bool final_) +AggregateDataBlockConverter::AggregateDataBlockConverter( + DB::Aggregator & aggregator_, DB::AggregatedDataVariantsPtr data_variants_, bool final_) : aggregator(aggregator_), data_variants(std::move(data_variants_)), final(final_) { if (data_variants->isTwoLevel()) - { buckets_num = DB::GlutenAggregatorUtil::getBucketsNum(*data_variants); - } else if (data_variants->size()) buckets_num = 1; else @@ -168,4 +188,109 @@ DB::Block AggregateDataBlockConverter::next() output_blocks.pop_front(); return block; } + +DB::Aggregator::Params AggregatorParamsHelper::buildParams( + const DB::ContextPtr & context, + const DB::Names & grouping_keys, + const DB::AggregateDescriptions & agg_descriptions, + Mode mode, + Algorithm algorithm) +{ + const auto & settings = context->getSettingsRef(); + size_t max_rows_to_group_by = mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_rows_to_group_by]); + + size_t group_by_two_level_threshold + = algorithm == Algorithm::GlutenGraceAggregate ? static_cast(settings[DB::Setting::group_by_two_level_threshold]) : 0; + size_t group_by_two_level_threshold_bytes = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::group_by_two_level_threshold_bytes])); + double max_bytes_ratio_before_external_group_by = algorithm == Algorithm::GlutenGraceAggregate + ? 0.0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0.0 : settings[DB::Setting::max_bytes_ratio_before_external_group_by]); + size_t max_bytes_before_external_group_by = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : (mode == Mode::PARTIAL_TO_FINISHED ? 0 : static_cast(settings[DB::Setting::max_bytes_before_external_group_by])); + bool empty_result_for_aggregation_by_empty_set = algorithm == Algorithm::GlutenGraceAggregate + ? false + : (mode == Mode::PARTIAL_TO_FINISHED ? false : static_cast(settings[DB::Setting::empty_result_for_aggregation_by_empty_set])); + DB::TemporaryDataOnDiskScopePtr tmp_data_scope = algorithm == Algorithm::GlutenGraceAggregate ? nullptr : context->getTempDataOnDisk(); + + size_t min_free_disk_space = algorithm == Algorithm::GlutenGraceAggregate + ? 0 + : static_cast(settings[DB::Setting::min_free_disk_space_for_temporary_data]); + bool compile_aggregate_expressions = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::compile_aggregate_expressions]; + size_t min_count_to_compile_aggregate_expression + = mode == Mode::PARTIAL_TO_FINISHED ? 0 : settings[DB::Setting::min_count_to_compile_aggregate_expression]; + size_t max_block_size = PODArrayUtil::adjustMemoryEfficientSize(settings[DB::Setting::max_block_size]); + bool enable_prefetch = mode != Mode::PARTIAL_TO_FINISHED; + bool only_merge = mode == Mode::PARTIAL_TO_FINISHED; + bool optimize_group_by_constant_keys + = mode == Mode::PARTIAL_TO_FINISHED ? false : settings[DB::Setting::optimize_group_by_constant_keys]; + + DB::Settings aggregate_settings{settings}; + aggregate_settings[DB::Setting::max_rows_to_group_by] = max_rows_to_group_by; + aggregate_settings[DB::Setting::max_bytes_ratio_before_external_group_by] = max_bytes_ratio_before_external_group_by; + aggregate_settings[DB::Setting::max_bytes_before_external_group_by] = max_bytes_before_external_group_by; + aggregate_settings[DB::Setting::min_free_disk_space_for_temporary_data] = min_free_disk_space; + aggregate_settings[DB::Setting::compile_aggregate_expressions] = compile_aggregate_expressions; + aggregate_settings[DB::Setting::min_count_to_compile_aggregate_expression] = min_count_to_compile_aggregate_expression; + aggregate_settings[DB::Setting::max_block_size] = max_block_size; + aggregate_settings[DB::Setting::enable_software_prefetch_in_aggregation] = enable_prefetch; + aggregate_settings[DB::Setting::optimize_group_by_constant_keys] = optimize_group_by_constant_keys; + return DB::Aggregator::Params{ + grouping_keys, + agg_descriptions, + /*overflow_row*/ false, + aggregate_settings[DB::Setting::max_rows_to_group_by], + aggregate_settings[DB::Setting::group_by_overflow_mode], + group_by_two_level_threshold, + group_by_two_level_threshold_bytes, + DB::Aggregator::Params::getMaxBytesBeforeExternalGroupBy( + aggregate_settings[DB::Setting::max_bytes_before_external_group_by], + aggregate_settings[DB::Setting::max_bytes_ratio_before_external_group_by]), + empty_result_for_aggregation_by_empty_set, + tmp_data_scope, + aggregate_settings[DB::Setting::max_threads], + aggregate_settings[DB::Setting::min_free_disk_space_for_temporary_data], + aggregate_settings[DB::Setting::compile_aggregate_expressions], + aggregate_settings[DB::Setting::min_count_to_compile_aggregate_expression], + aggregate_settings[DB::Setting::max_block_size], + aggregate_settings[DB::Setting::enable_software_prefetch_in_aggregation], + only_merge, + aggregate_settings[DB::Setting::optimize_group_by_constant_keys], + aggregate_settings[DB::Setting::min_hit_rate_to_use_consecutive_keys_optimization], + {}}; +} + + +#define COMPARE_FIELD(field) \ + if (lhs.field != rhs.field) \ + { \ + LOG_ERROR(getLogger("AggregatorParamsHelper"), "Aggregator::Params field " #field " is not equal. {}/{}", lhs.field, rhs.field); \ + return false; \ + } +bool AggregatorParamsHelper::compare(const DB::Aggregator::Params & lhs, const DB::Aggregator::Params & rhs) +{ + COMPARE_FIELD(overflow_row); + COMPARE_FIELD(max_rows_to_group_by); + COMPARE_FIELD(group_by_overflow_mode); + COMPARE_FIELD(group_by_two_level_threshold); + COMPARE_FIELD(group_by_two_level_threshold_bytes); + COMPARE_FIELD(max_bytes_before_external_group_by); + COMPARE_FIELD(empty_result_for_aggregation_by_empty_set); + COMPARE_FIELD(max_threads); + COMPARE_FIELD(min_free_disk_space); + COMPARE_FIELD(compile_aggregate_expressions); + if ((lhs.tmp_data_scope == nullptr) != (rhs.tmp_data_scope == nullptr)) + { + LOG_ERROR(getLogger("AggregatorParamsHelper"), "Aggregator::Params field tmp_data_scope is not equal."); + return false; + } + COMPARE_FIELD(min_count_to_compile_aggregate_expression); + COMPARE_FIELD(enable_prefetch); + COMPARE_FIELD(only_merge); + COMPARE_FIELD(optimize_group_by_constant_keys); + COMPARE_FIELD(min_hit_rate_to_use_consecutive_keys_optimization); + return true; +} } diff --git a/cpp-ch/local-engine/Common/AggregateUtil.h b/cpp-ch/local-engine/Common/AggregateUtil.h index b14cd59c5490..8fd36987ac8c 100644 --- a/cpp-ch/local-engine/Common/AggregateUtil.h +++ b/cpp-ch/local-engine/Common/AggregateUtil.h @@ -25,7 +25,8 @@ class GlutenAggregatorUtil { public: static Int32 getBucketsNum(AggregatedDataVariants & data_variants); - static std::optional safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket); + static std::optional + safeConvertOneBucketToBlock(Aggregator & aggregator, AggregatedDataVariants & variants, Arena * arena, bool final, Int32 bucket); static void safeReleaseOneBucket(AggregatedDataVariants & variants, Int32 bucket); }; } @@ -41,6 +42,7 @@ class AggregateDataBlockConverter ~AggregateDataBlockConverter() = default; bool hasNext(); DB::Block next(); + private: DB::Aggregator & aggregator; DB::AggregatedDataVariantsPtr data_variants; @@ -50,4 +52,31 @@ class AggregateDataBlockConverter Int32 current_bucket = 0; DB::BlocksList output_blocks; }; + +class AggregatorParamsHelper +{ +public: + enum class Algorithm + { + GlutenGraceAggregate, + CHTwoStageAggregate + }; + enum class Mode + { + INIT_TO_PARTIAL, + INIT_TO_COMPLETED, + PARTIAL_TO_PARTIAL, + PARTIAL_TO_FINISHED, + }; + + // for using grace aggregating, never enable ch spill, otherwise there will be data lost. + static DB::Aggregator::Params buildParams( + const DB::ContextPtr & context, + const DB::Names & grouping_keys, + const DB::AggregateDescriptions & agg_descriptions, + Mode mode, + Algorithm algorithm = Algorithm::GlutenGraceAggregate); + static bool compare(const DB::Aggregator::Params & lhs, const DB::Aggregator::Params & rhs); +}; + } diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp new file mode 100644 index 000000000000..acefad0aea2a --- /dev/null +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.cpp @@ -0,0 +1,175 @@ +/* + * 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. + */ + +#include "ArrayJoinHelper.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace +{ +extern const int LOGICAL_ERROR; +} + +namespace Setting +{ +extern const SettingsUInt64 max_block_size; +} +} + +namespace local_engine +{ +namespace ArrayJoinHelper +{ +const DB::ActionsDAG::Node * findArrayJoinNode(const DB::ActionsDAG & actions_dag) +{ + const DB::ActionsDAG::Node * array_join_node = nullptr; + const auto & nodes = actions_dag.getNodes(); + for (const auto & node : nodes) + { + if (node.type == DB::ActionsDAG::ActionType::ARRAY_JOIN) + { + if (array_join_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect single ARRAY JOIN node in generate rel"); + array_join_node = &node; + } + } + return array_join_node; +} + +struct SplittedActionsDAGs +{ + DB::ActionsDAG before_array_join; /// Optional + DB::ActionsDAG array_join; + DB::ActionsDAG after_array_join; /// Optional +}; + +/// Split actions_dag of generate rel into 3 parts: before array join + during array join + after array join +static SplittedActionsDAGs splitActionsDAGInGenerate(const DB::ActionsDAG & actions_dag) +{ + SplittedActionsDAGs res; + + auto array_join_node = findArrayJoinNode(actions_dag); + std::unordered_set first_split_nodes(array_join_node->children.begin(), array_join_node->children.end()); + auto first_split_result = actions_dag.split(first_split_nodes); + res.before_array_join = std::move(first_split_result.first); + + array_join_node = findArrayJoinNode(first_split_result.second); + std::unordered_set second_split_nodes = {array_join_node}; + auto second_split_result = first_split_result.second.split(second_split_nodes); + res.array_join = std::move(second_split_result.first); + second_split_result.second.removeUnusedActions(); + res.after_array_join = std::move(second_split_result.second); + return res; +} + +DB::ActionsDAG applyArrayJoinOnOneColumn(const DB::Block & header, size_t column_index) +{ + auto arrayColumn = header.getByPosition(column_index); + if (!typeid_cast(arrayColumn.type.get())) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect array column in array join"); + DB::ActionsDAG actions_dag(header.getColumnsWithTypeAndName()); + const auto * array_column_node = actions_dag.getInputs()[column_index]; + auto array_join_name = array_column_node->result_name; + const auto * array_join_node = &actions_dag.addArrayJoin(*array_column_node, array_join_name); + actions_dag.addOrReplaceInOutputs(*array_join_node); + return std::move(actions_dag); +} + + +std::vector +addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::ActionsDAG & actions_dag, bool is_left) +{ + auto logger = getLogger("ArrayJoinHelper"); + std::vector steps; + if (findArrayJoinNode(actions_dag)) + { + /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size + /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls + LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); + auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); + LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); + LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); + + auto ignore_actions_dag = [](const DB::ActionsDAG & actions_dag_) -> bool + { + /* + We should ignore actions_dag like: + 0 : INPUT () (no column) String a + 1 : INPUT () (no column) String b + Output nodes: 0, 1 + */ + return actions_dag_.getOutputs().size() == actions_dag_.getNodes().size() + && actions_dag_.getInputs().size() == actions_dag_.getNodes().size(); + }; + + /// Pre-projection before array join + if (!ignore_actions_dag(splitted_actions_dags.before_array_join)) + { + auto step_before_array_join + = std::make_unique(plan.getCurrentHeader(), std::move(splitted_actions_dags.before_array_join)); + step_before_array_join->setStepDescription("Pre-projection In Generate"); + steps.emplace_back(step_before_array_join.get()); + plan.addStep(std::move(step_before_array_join)); + // LOG_DEBUG(logger, "plan1:{}", PlanUtil::explainPlan(*query_plan)); + } + + /// ARRAY JOIN + DB::Names array_joined_columns{findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; + DB::ArrayJoin array_join; + array_join.columns = std::move(array_joined_columns); + array_join.is_left = is_left; + auto array_join_step = std::make_unique( + plan.getCurrentHeader(), std::move(array_join), false, context->getSettingsRef()[DB::Setting::max_block_size]); + array_join_step->setStepDescription("ARRAY JOIN In Generate"); + steps.emplace_back(array_join_step.get()); + plan.addStep(std::move(array_join_step)); + // LOG_DEBUG(logger, "plan2:{}", PlanUtil::explainPlan(*query_plan)); + + /// Post-projection after array join(Optional) + if (!ignore_actions_dag(splitted_actions_dags.after_array_join)) + { + auto step_after_array_join + = std::make_unique(plan.getCurrentHeader(), std::move(splitted_actions_dags.after_array_join)); + step_after_array_join->setStepDescription("Post-projection In Generate"); + steps.emplace_back(step_after_array_join.get()); + plan.addStep(std::move(step_after_array_join)); + // LOG_DEBUG(logger, "plan3:{}", PlanUtil::explainPlan(*query_plan)); + } + } + else + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Expect array join node in actions_dag"); + } + + return steps; +} + + +} +} // namespace local_engine \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/ArrayJoinHelper.h b/cpp-ch/local-engine/Common/ArrayJoinHelper.h new file mode 100644 index 000000000000..a4b582b2f0c0 --- /dev/null +++ b/cpp-ch/local-engine/Common/ArrayJoinHelper.h @@ -0,0 +1,39 @@ +/* + * 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. + */ +#pragma once +#include + +namespace DB +{ +class IQueryPlanStep; +class QueryPlan; +} + +namespace local_engine +{ +namespace ArrayJoinHelper +{ +// apply array join on one column to flatten the array column +DB::ActionsDAG applyArrayJoinOnOneColumn(const DB::Block & header, size_t column_index); + +const DB::ActionsDAG::Node * findArrayJoinNode(const DB::ActionsDAG & actions_dag); + +// actions_dag is a actions dag that contains the array join node, if not, the plan will not be changed. +// return the steps that are added to the plan. +std::vector addArrayJoinStep(DB::ContextPtr context, DB::QueryPlan & plan, const DB::ActionsDAG & actions_dag, bool is_left); +} // namespace ArrayJoinHelper +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/BlockIterator.cpp b/cpp-ch/local-engine/Common/BlockIterator.cpp index 464701893207..150b5c908c51 100644 --- a/cpp-ch/local-engine/Common/BlockIterator.cpp +++ b/cpp-ch/local-engine/Common/BlockIterator.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes namespace local_engine { -void local_engine::BlockIterator::checkNextValid() +void local_engine::BlockIterator::checkNextValid() const { if (consumed) { diff --git a/cpp-ch/local-engine/Common/BlockIterator.h b/cpp-ch/local-engine/Common/BlockIterator.h index a57935665e3e..4b9d6abcc30e 100644 --- a/cpp-ch/local-engine/Common/BlockIterator.h +++ b/cpp-ch/local-engine/Common/BlockIterator.h @@ -23,7 +23,7 @@ namespace local_engine class BlockIterator { protected: - void checkNextValid(); + void checkNextValid() const; // make current block available void produce(); // consume current block diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.cpp b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp index f6dcd0959225..35890cc78131 100644 --- a/cpp-ch/local-engine/Common/BlockTypeUtils.cpp +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.cpp @@ -39,18 +39,12 @@ DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type) { if (nested_type->isLowCardinalityNullable()) return nested_type; - else if (!nested_type->lowCardinality()) - return std::make_shared(nested_type); - else + if (nested_type->lowCardinality()) return std::make_shared( std::make_shared(dynamic_cast(*nested_type).getDictionaryType())); - } - - - if (nullable && !nested_type->isNullable()) return std::make_shared(nested_type); - else - return nested_type; + } + return nested_type; } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/BlockTypeUtils.h b/cpp-ch/local-engine/Common/BlockTypeUtils.h index af01b528d6e9..64032af70f68 100644 --- a/cpp-ch/local-engine/Common/BlockTypeUtils.h +++ b/cpp-ch/local-engine/Common/BlockTypeUtils.h @@ -73,14 +73,18 @@ inline DB::DataTypePtr DATE() return std::make_shared(); } -inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data_) +inline DB::Block makeBlockHeader(const DB::ColumnsWithTypeAndName & data) { - return DB::Block(data_); + return DB::Block(data); } DB::NamesAndTypesList blockToNameAndTypeList(const DB::Block & header); DB::DataTypePtr wrapNullableType(bool nullable, DB::DataTypePtr nested_type); +inline DB::DataTypePtr wrapNullableType(DB::DataTypePtr nested_type) +{ + return wrapNullableType(true, nested_type); +} inline DB::DataTypePtr wrapNullableType(const substrait::Type_Nullability nullable, const DB::DataTypePtr & nested_type) { return wrapNullableType(nullable == substrait::Type_Nullability_NULLABILITY_NULLABLE, nested_type); diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index 369dceb61afc..310b39d3e594 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include #include #include @@ -75,10 +76,19 @@ namespace DB { +namespace ServerSetting +{ +extern const ServerSettingsString primary_index_cache_policy; +extern const ServerSettingsUInt64 primary_index_cache_size; +extern const ServerSettingsDouble primary_index_cache_size_ratio; +} namespace Setting { extern const SettingsUInt64 prefer_external_sort_block_bytes; extern const SettingsUInt64 max_bytes_before_external_sort; +extern const SettingsBool query_plan_merge_filters; +extern const SettingsBool compile_expressions; +extern const SettingsShortCircuitFunctionEvaluation short_circuit_function_evaluation; } namespace ErrorCodes { @@ -86,6 +96,14 @@ extern const int BAD_ARGUMENTS; extern const int UNKNOWN_TYPE; extern const int CANNOT_PARSE_PROTOBUF_SCHEMA; } + +namespace ServerSetting +{ +extern const ServerSettingsUInt64 max_thread_pool_size; +extern const ServerSettingsUInt64 thread_pool_queue_size; +extern const ServerSettingsUInt64 max_io_thread_pool_size; +extern const ServerSettingsUInt64 io_thread_pool_queue_size; +} } namespace local_engine @@ -480,21 +498,7 @@ std::optional NestedColumnExtractHelper::extractColum const DB::ColumnWithTypeAndName * NestedColumnExtractHelper::findColumn(const DB::Block & in_block, const std::string & name) const { - if (case_insentive) - { - std::string final_name = name; - boost::to_lower(final_name); - const auto & cols = in_block.getColumnsWithTypeAndName(); - auto found = std::find_if(cols.begin(), cols.end(), [&](const auto & column) { return boost::iequals(column.name, name); }); - if (found == cols.end()) - return nullptr; - return &*found; - } - - const auto * col = in_block.findByName(name); - if (col) - return col; - return nullptr; + return in_block.findByName(name, case_insentive); } const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( @@ -583,7 +587,7 @@ std::vector BackendInitializerUtil::wrapDiskPathConfig( return changed_paths; } -DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map & spark_conf_map) +DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const SparkConfigs::ConfigMap & spark_conf_map) { DB::Context::ConfigurationPtr config; @@ -618,7 +622,7 @@ DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map< if (spark_conf_map.contains(GLUTEN_TASK_OFFHEAP)) config->setString(MemoryConfig::CH_TASK_MEMORY, spark_conf_map.at(GLUTEN_TASK_OFFHEAP)); - const bool use_current_directory_as_tmp = config->getBool("use_current_directory_as_tmp", false); + const bool use_current_directory_as_tmp = config->getBool(PathConfig::USE_CURRENT_DIRECTORY_AS_TMP, false); char buffer[PATH_MAX]; if (use_current_directory_as_tmp && getcwd(buffer, sizeof(buffer)) != nullptr) wrapDiskPathConfig(String(buffer), "", *config); @@ -636,7 +640,7 @@ DB::Context::ConfigurationPtr BackendInitializerUtil::initConfig(const std::map< return config; } -String BackendInitializerUtil::tryGetConfigFile(const std::map & spark_conf_map) +String BackendInitializerUtil::tryGetConfigFile(const SparkConfigs::ConfigMap & spark_conf_map) { if (spark_conf_map.contains(CH_RUNTIME_CONFIG_FILE)) return spark_conf_map.at(CH_RUNTIME_CONFIG_FILE); @@ -701,7 +705,7 @@ DB::Field BackendInitializerUtil::toField(const String & key, const String & val return DB::Field(value); } -void BackendInitializerUtil::initSettings(const std::map & spark_conf_map, DB::Settings & settings) +void BackendInitializerUtil::initSettings(const SparkConfigs::ConfigMap & spark_conf_map, DB::Settings & settings) { /// Initialize default setting. settings.set("date_time_input_format", "best_effort"); @@ -712,6 +716,17 @@ void BackendInitializerUtil::initSettings(const std::mapmakeGlobalContext(); global_context->setConfig(config); - auto getDefaultPath = [config] -> auto + auto tmp_path = config->getString("tmp_path", PathConfig::DEFAULT_TEMP_FILE_PATH); + if(config->getBool(PathConfig::USE_CURRENT_DIRECTORY_AS_TMP, false)) { - const bool use_current_directory_as_tmp = config->getBool("use_current_directory_as_tmp", false); char buffer[PATH_MAX]; - if (use_current_directory_as_tmp && getcwd(buffer, sizeof(buffer)) != nullptr) - return std::string(buffer) + "/tmp/libch"; - else - return std::string("/tmp/libch"); + if (getcwd(buffer, sizeof(buffer)) != nullptr) + tmp_path = std::string(buffer) + tmp_path; }; - global_context->setTemporaryStoragePath(config->getString("tmp_path", getDefaultPath()), 0); + global_context->setTemporaryStoragePath(tmp_path, 0); global_context->setPath(config->getString("path", "/")); String uncompressed_cache_policy = config->getString("uncompressed_cache_policy", DEFAULT_UNCOMPRESSED_CACHE_POLICY); @@ -833,10 +856,16 @@ void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) size_t mark_cache_size = config->getUInt64("mark_cache_size", DEFAULT_MARK_CACHE_MAX_SIZE); double mark_cache_size_ratio = config->getDouble("mark_cache_size_ratio", DEFAULT_MARK_CACHE_SIZE_RATIO); if (!mark_cache_size) - LOG_ERROR(&Poco::Logger::get("CHUtil"), "Too low mark cache size will lead to severe performance degradation."); - + LOG_ERROR(log, "Mark cache is disabled, it will lead to severe performance degradation."); + LOG_INFO(log, "mark cache size to {}.", formatReadableSizeWithBinarySuffix(mark_cache_size)); global_context->setMarkCache(mark_cache_policy, mark_cache_size, mark_cache_size_ratio); + String primary_index_cache_policy = server_settings[ServerSetting::primary_index_cache_policy]; + size_t primary_index_cache_size = server_settings[ServerSetting::primary_index_cache_size]; + double primary_index_cache_size_ratio = server_settings[ServerSetting::primary_index_cache_size_ratio]; + LOG_INFO(log, "Primary index cache size to {}.", formatReadableSizeWithBinarySuffix(primary_index_cache_size)); + global_context->setPrimaryIndexCache(primary_index_cache_policy, primary_index_cache_size, primary_index_cache_size_ratio); + String index_uncompressed_cache_policy = config->getString("index_uncompressed_cache_policy", DEFAULT_INDEX_UNCOMPRESSED_CACHE_POLICY); size_t index_uncompressed_cache_size @@ -877,6 +906,7 @@ void BackendInitializerUtil::applyGlobalConfigAndSettings(const DB::Context::Con extern void registerAggregateFunctionCombinatorPartialMerge(AggregateFunctionCombinatorFactory &); extern void registerAggregateFunctionsBloomFilter(AggregateFunctionFactory &); extern void registerAggregateFunctionSparkAvg(AggregateFunctionFactory &); +extern void registerAggregateFunctionRowNumGroup(AggregateFunctionFactory &); extern void registerFunctions(FunctionFactory &); void registerAllFunctions() @@ -887,6 +917,7 @@ void registerAllFunctions() auto & agg_factory = AggregateFunctionFactory::instance(); registerAggregateFunctionsBloomFilter(agg_factory); registerAggregateFunctionSparkAvg(agg_factory); + registerAggregateFunctionRowNumGroup(agg_factory); { /// register aggregate function combinators from local_engine auto & factory = AggregateFunctionCombinatorFactory::instance(); @@ -931,7 +962,7 @@ void BackendInitializerUtil::initCompiledExpressionCache(DB::Context::Configurat #endif } -void BackendInitializerUtil::initBackend(const std::map & spark_conf_map) +void BackendInitializerUtil::initBackend(const SparkConfigs::ConfigMap & spark_conf_map) { DB::Context::ConfigurationPtr config = initConfig(spark_conf_map); @@ -972,7 +1003,12 @@ void BackendInitializerUtil::initBackend(const std::mapgetUInt("max_active_parts_loading_thread_pool_size", 64); DB::getActivePartsLoadingThreadPool().initialize( @@ -1071,8 +1107,12 @@ JoinUtil::getJoinKindAndStrictness(substrait::JoinRel_JoinType join_type, bool i return {DB::JoinKind::Left, DB::JoinStrictness::Any}; return {DB::JoinKind::Left, DB::JoinStrictness::Semi}; } + case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI: + return {DB::JoinKind::Right, DB::JoinStrictness::Semi}; case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT_ANTI: return {DB::JoinKind::Left, DB::JoinStrictness::Anti}; + case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI: + return {DB::JoinKind::Right, DB::JoinStrictness::Anti}; case substrait::JoinRel_JoinType_JOIN_TYPE_LEFT: return {DB::JoinKind::Left, DB::JoinStrictness::All}; case substrait::JoinRel_JoinType_JOIN_TYPE_RIGHT: diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index c3d067f245f0..a5fb24f6afee 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -27,6 +27,7 @@ #include #include #include +#include namespace DB { @@ -39,6 +40,7 @@ namespace local_engine static const String MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE = "mergetree.insert_without_local_storage"; static const String MERGETREE_MERGE_AFTER_INSERT = "mergetree.merge_after_insert"; static const std::string DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = "spark.sql.decimalOperations.allowPrecisionLoss"; +static const std::string TIMER_PARSER_POLICY = "spark.sql.legacy.timeParserPolicy"; static const std::unordered_set BOOL_VALUE_SETTINGS{ MERGETREE_MERGE_AFTER_INSERT, MERGETREE_INSERT_WITHOUT_LOCAL_STORAGE, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS}; @@ -158,8 +160,8 @@ class BackendInitializerUtil /// Initialize two kinds of resources /// 1. global level resources like global_context/shared_context, notice that they can only be initialized once in process lifetime /// 2. session level resources like settings/configs, they can be initialized multiple times following the lifetime of executor/driver - static void initBackend(const std::map & spark_conf_map); - static void initSettings(const std::map & spark_conf_map, DB::Settings & settings); + static void initBackend(const SparkConfigs::ConfigMap & spark_conf_map); + static void initSettings(const SparkConfigs::ConfigMap & spark_conf_map, DB::Settings & settings); inline static const String CH_BACKEND_PREFIX = "spark.gluten.sql.columnar.backend.ch"; @@ -199,8 +201,8 @@ class BackendInitializerUtil friend class BackendFinalizerUtil; friend class JNIUtils; - static DB::Context::ConfigurationPtr initConfig(const std::map & spark_conf_map); - static String tryGetConfigFile(const std::map & spark_conf_map); + static DB::Context::ConfigurationPtr initConfig(const SparkConfigs::ConfigMap & spark_conf_map); + static String tryGetConfigFile(const SparkConfigs::ConfigMap & spark_conf_map); static void initLoggers(DB::Context::ConfigurationPtr config); static void initEnvs(DB::Context::ConfigurationPtr config); diff --git a/cpp-ch/local-engine/Common/DebugUtils.cpp b/cpp-ch/local-engine/Common/DebugUtils.cpp index c144a1384ee2..8cda6734670e 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.cpp +++ b/cpp-ch/local-engine/Common/DebugUtils.cpp @@ -19,91 +19,350 @@ #include #include #include -#include -#include #include -#include #include #include +#include +#include +#include +#include +#include +#include +#include + +namespace pb_util = google::protobuf::util; namespace debug { -void headBlock(const DB::Block & block, size_t count) +namespace Utils +{ + +/** + * Return the number of half widths in a given string. Note that a full width character + * occupies two half widths. + * + * For a string consisting of 1 million characters, the execution of this method requires + * about 50ms. + */ +static size_t stringHalfWidth(const std::string & str) +{ + //TODO: Implement this method + return str.size(); +} + +/** + *

Left pad a String with spaces (' ').

+ * + *

The String is padded to the size of {@code size}.

+ * + *
+ * StringUtils.leftPad(null, *)   = null
+ * StringUtils.leftPad("", 3)     = "   "
+ * StringUtils.leftPad("bat", 3)  = "bat"
+ * StringUtils.leftPad("bat", 5)  = "  bat"
+ * StringUtils.leftPad("bat", 1)  = "bat"
+ * StringUtils.leftPad("bat", -1) = "bat"
+ * 
+ * + * @param str the String to pad out, may be null + * @param size the size to pad to + * @return left padded String or original String if no padding is necessary, + * {@code null} if null String input + */ +static std::string leftPad(const std::string & str, int totalWidth) +{ + std::stringstream ss; + ss << std::setw(totalWidth) << std::setfill(' ') << str; + return ss.str(); +} + +/** + *

Right pad a String with spaces (' ').

+ * + *

The String is padded to the size of {@code size}.

+ * + *
+ * StringUtils.rightPad(null, *)   = null
+ * StringUtils.rightPad("", 3)     = "   "
+ * StringUtils.rightPad("bat", 3)  = "bat"
+ * StringUtils.rightPad("bat", 5)  = "bat  "
+ * StringUtils.rightPad("bat", 1)  = "bat"
+ * StringUtils.rightPad("bat", -1) = "bat"
+ * 
+ * + * @param str the String to pad out, may be null + * @param totalWidth the size to pad to + * @param padChar the character to pad with + * @param size the size to pad to + * @return right padded String or original String if no padding is necessary, + * {@code null} if null String input + */ +static std::string rightPad(const std::string & str, int totalWidth, char padChar = ' ') +{ + std::stringstream ss; + ss << str << std::setw(totalWidth - str.size()) << std::setfill(padChar) << ""; + return ss.str(); +} + +static std::string truncate(const std::string & str, size_t width) +{ + if (str.size() <= width) + return str; + return str.substr(0, width - 3) + "..."; +} + +using NameAndColumn = std::pair; +using NameAndColumns = std::vector; + +/** + * Get rows represented in Sequence by specific truncate and vertical requirement. + * + * @param block Columns to show + * @param numRows Number of rows to return + * @param truncate If set to more than 0, truncates strings to `truncate` characters and + * all cells will be aligned right. + */ +static std::vector> getRows(const NameAndColumns & block, size_t numRows, size_t truncate) { - std::cout << "============Block============" << std::endl; - std::cout << block.dumpStructure() << std::endl; - // print header - for (const auto & name : block.getNames()) - std::cout << name << "\t"; - std::cout << std::endl; - - // print rows - for (size_t row = 0; row < std::min(count, block.rows()); ++row) + std::vector> results; + results.reserve(numRows); + results.emplace_back(std::vector()); + auto & headRow = results.back(); + + for (const auto & column : block) { - for (size_t column = 0; column < block.columns(); ++column) - { - const auto type = block.getByPosition(column).type; - auto col = block.getByPosition(column).column; + const auto & name = column.first; + headRow.emplace_back(debug::Utils::truncate(name, truncate)); + } - if (column > 0) - std::cout << "\t"; - DB::WhichDataType which(type); + auto getDataType = [](const DB::IColumn * col) + { + if (const auto * column_nullable = DB::checkAndGetColumn(col)) + return column_nullable->getNestedColumn().getDataType(); + return col->getDataType(); + }; + + for (size_t row = 0; row < numRows - 1; ++row) + { + results.emplace_back(std::vector()); + auto & currentRow = results.back(); + currentRow.reserve(block.size()); + + for (const auto & column : block) + { + const auto * const col = column.second.get(); + DB::WhichDataType which(getDataType(col)); if (which.isAggregateFunction()) - std::cout << "Nan"; - else if (col->isNullAt(row)) - std::cout << "null"; + currentRow.emplace_back("Nan"); else - std::cout << toString((*col)[row]); + { + if (col->isNullAt(row)) + currentRow.emplace_back("null"); + else + { + std::string str = DB::toString((*col)[row]); + currentRow.emplace_back(Utils::truncate(str, truncate)); + } + } } - std::cout << std::endl; } + return results; } -String printBlock(const DB::Block & block, size_t count) +static std::string showString(const NameAndColumns & block, size_t numRows, size_t truncate, bool vertical) { - std::ostringstream ss; - ss << std::string("============Block============\n"); - ss << block.dumpStructure() << String("\n"); - // print header - for (const auto & name : block.getNames()) - ss << name << std::string("\t"); - ss << std::string("\n"); - - // print rows - for (size_t row = 0; row < std::min(count, block.rows()); ++row) + numRows = std::min(numRows, block[0].second->size()); + bool hasMoreData = block[0].second->size() > numRows; + // Get rows represented by vector[vector[String]], we may get one more line if it has more data. + std::vector> rows = getRows(block, numRows + 1, truncate); + + size_t numCols = block.size(); + // We set a minimum column width at '3' + constexpr size_t minimumColWidth = 3; + + std::stringstream sb; + + if (!vertical) { - for (size_t column = 0; column < block.columns(); ++column) + // Initialise the width of each column to a minimum value + std::vector colWidths(numCols, minimumColWidth); + + // Compute the width of each column + for (const auto & row : rows) + for (size_t i = 0; i < row.size(); ++i) + colWidths[i] = std::max(colWidths[i], stringHalfWidth(row[i])); + + std::vector> paddedRows; + for (const auto & row : rows) { - const auto type = block.getByPosition(column).type; - auto col = block.getByPosition(column).column; + std::vector paddedRow; + for (size_t i = 0; i < row.size(); ++i) + if (truncate > 0) + paddedRow.push_back(leftPad(row[i], colWidths[i] - stringHalfWidth(row[i]) + row[i].size())); + else + paddedRow.push_back(rightPad(row[i], colWidths[i] - stringHalfWidth(row[i]) + row[i].size())); + paddedRows.push_back(paddedRow); + } - if (column > 0) - ss << std::string("\t"); - DB::WhichDataType which(type); - if (which.isAggregateFunction()) - ss << std::string("Nan"); - else if (col->isNullAt(row)) - ss << std::string("null"); - else - ss << toString((*col)[row]); + // Create SeparateLine + std::stringstream sep; + for (int width : colWidths) + sep << "+" << std::string(width, '-'); + sep << "+\n"; + + // column names + sb << sep.str(); + for (const auto & cell : paddedRows[0]) + sb << "|" << cell; + sb << "|\n" << sep.str(); + + // data + for (size_t i = 1; i < paddedRows.size(); ++i) + { + for (const auto & cell : paddedRows[i]) + sb << "|" << cell; + sb << "|\n"; } - ss << std::string("\n"); + sb << sep.str(); } - return ss.str(); + else + { + // Extended display mode enabled + const std::vector & fieldNames = rows[0]; + auto dataRowsBegin = [&]() { return rows.begin() + 1; }; + + // Compute the width of field name and data columns + size_t fieldNameColWidth = minimumColWidth; + for (const auto & fieldName : fieldNames) + fieldNameColWidth = std::max(fieldNameColWidth, Utils::stringHalfWidth(fieldName)); + + size_t dataColWidth = minimumColWidth; + + + for (auto dataRowIter = dataRowsBegin(); dataRowIter != rows.end(); ++dataRowIter) + { + const auto & row = *dataRowIter; + size_t maxWidth = 0; + for (const auto & cell : row) + maxWidth = std::max(maxWidth, stringHalfWidth(cell)); + dataColWidth = std::max(dataColWidth, maxWidth); + } + + // + for (auto dataRowIter = dataRowsBegin(); dataRowIter != rows.end(); ++dataRowIter) + { + // create row header + std::string rowHeader = "-RECORD " + std::to_string(rows.end() - dataRowIter); + rowHeader = rightPad(rowHeader, fieldNameColWidth + dataColWidth + 5, '-'); + sb << rowHeader << "\n"; + + // process each cell in the row + const auto & row = *dataRowIter; + for (size_t j = 0; j < row.size(); j++) + { + const std::string & cell = row[j]; + const std::string & fieldName = fieldNames[j]; + std::string paddedFieldName = rightPad(fieldName, fieldNameColWidth - stringHalfWidth(fieldName) + fieldName.length()); + std::string paddedData = rightPad(cell, dataColWidth - stringHalfWidth(cell) + cell.length()); + sb << " " << paddedFieldName << " | " << paddedData << " \n"; + } + sb << "\n"; + } + } + + // Print a footer + if (vertical && block[0].second->empty()) + { + // In a vertical mode, print an empty row set explicitly + sb << "(0 rows)" << std::endl; + } + else if (hasMoreData) + { + // For Data that has more than "numRows" records + const char * rowsString = (numRows == 1) ? "row" : "rows"; + sb << "only showing top " << numRows << " " << rowsString << std::endl; + } + return sb.str(); } +} // namespace Utils -void headColumn(const DB::ColumnPtr & column, size_t count) +/// + +void dumpPlan(DB::QueryPlan & plan, const char * type, bool force, LoggerPtr logger) { - std::cout << "============Column============" << std::endl; + if (!logger) + { + logger = getLogger("SerializedPlanParser"); + if (!logger) + return; + } + + if (!force && !logger->debug()) + return; + + auto out = local_engine::PlanUtil::explainPlan(plan); + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; + if (force) // force + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, out); + else + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, out); +} + +void dumpMessage(const google::protobuf::Message & message, const char * type, bool force, LoggerPtr logger) +{ + if (!logger) + { + logger = getLogger("SubstraitPlan"); + if (!logger) + return; + } + + if (!force && !logger->debug()) + return; + pb_util::JsonOptions options; + std::string json; + if (auto s = MessageToJsonString(message, &json, options); !s.ok()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); - // print header - std::cout << column->getName() << "\t"; - std::cout << std::endl; + auto task_id = local_engine::QueryContext::instance().currentTaskIdOrEmpty(); + task_id = task_id.empty() ? "" : "(" + task_id + ")"; + if (force) // force + LOG_ERROR(logger, "{}{} =>\n{}", type, task_id, json); + else + LOG_DEBUG(logger, "{}{} =>\n{}", type, task_id, json); +} - // print rows - for (size_t row = 0; row < std::min(count, column->size()); ++row) - std::cout << toString((*column)[row]) << std::endl; +void headBlock(const DB::Block & block, size_t count) +{ + std::cerr << showString(block, count) << std::endl; } +void headColumn(const DB::ColumnPtr & column, size_t count) +{ + std::cerr << Utils::showString({{"Column", column}}, count, 20, false) << std::endl; +} + +/** + * Compose the string representing rows for output + * + * @param block Block to show + * @param numRows Number of rows to show + * @param truncate If set to more than 0, truncates strings to `truncate` characters and + * all cells will be aligned right. + * @param vertical If set to true, prints output rows vertically (one line per column value). + */ + +std::string showString(const DB::Block & block, size_t numRows, size_t truncate, bool vertical) +{ + std::vector columns = block.getColumnsWithTypeAndName(); + Utils::NameAndColumns name_and_columns; + name_and_columns.reserve(columns.size()); + std::ranges::transform( + columns, + std::back_inserter(name_and_columns), + [](const DB::ColumnWithTypeAndName & col) { return std::make_pair(col.name, col.column); }); + return Utils::showString(name_and_columns, numRows, truncate, vertical); } +} \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/DebugUtils.h b/cpp-ch/local-engine/Common/DebugUtils.h index cc0ecdc59dfe..7b9fb637dcc6 100644 --- a/cpp-ch/local-engine/Common/DebugUtils.h +++ b/cpp-ch/local-engine/Common/DebugUtils.h @@ -18,10 +18,26 @@ #include +namespace google::protobuf +{ +class Message; +} +namespace DB +{ +class QueryPlan; +} namespace debug { -void headBlock(const DB::Block & block, size_t count = 10); -String printBlock(const DB::Block & block, size_t count = 10); +void dumpPlan(DB::QueryPlan & plan, const char * type = "clickhouse plan", bool force = false, LoggerPtr = nullptr); +void dumpMessage(const google::protobuf::Message & message, const char * type, bool force = false, LoggerPtr = nullptr); + +void headBlock(const DB::Block & block, size_t count = 10); void headColumn(const DB::ColumnPtr & column, size_t count = 10); +std::string showString(const DB::Block & block, size_t numRows = 20, size_t truncate = 20, bool vertical = false); +inline std::string verticalShowString(const DB::Block & block, size_t numRows = 20, size_t truncate = 20) +{ + return showString(block, numRows, truncate, true); +} + } diff --git a/cpp-ch/local-engine/Common/GlutenConfig.cpp b/cpp-ch/local-engine/Common/GlutenConfig.cpp index 44d77cf372cb..75bb41f10d98 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.cpp +++ b/cpp-ch/local-engine/Common/GlutenConfig.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include namespace local_engine @@ -29,28 +30,26 @@ namespace local_engine void SparkConfigs::updateConfig(const DB::ContextMutablePtr & context, std::string_view plan) { - std::map spark_conf_map = load(plan); - // configs cannot be updated per query - // settings can be updated per query - auto settings = context->getSettingsCopy(); // make a copy - - // TODO: Remove BackendInitializerUtil::initSettings - BackendInitializerUtil::initSettings(spark_conf_map, settings); - context->setSettings(settings); + update( + plan, + [&](const ConfigMap & spark_conf_map) + { + // configs cannot be updated per query + // settings can be updated per query + auto settings = context->getSettingsCopy(); // make a copy + + // TODO: Remove BackendInitializerUtil::initSettings + BackendInitializerUtil::initSettings(spark_conf_map, settings); + context->setSettings(settings); + }); } -std::map SparkConfigs::load(std::string_view plan, bool processStart) +void SparkConfigs::update(std::string_view plan, const std::function & callback, bool processStart) { - std::map configs{}; auto configMaps = local_engine::BinaryToMessage(plan); - if (!processStart) - logDebugMessage(configMaps, "Update Config Map Plan"); - - for (const auto & pair : configMaps.configs()) - configs.emplace(pair.first, pair.second); - - return configs; + debug::dumpMessage(configMaps, "Update Config Map Plan"); + callback(configMaps.configs()); } MemoryConfig MemoryConfig::loadFromContext(const DB::ContextPtr & context) @@ -74,6 +73,7 @@ GraceMergingAggregateConfig GraceMergingAggregateConfig::loadFromContext(const D = context->getConfigRef().getUInt64(MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET, 1_MiB); config.max_allowed_memory_usage_ratio_for_aggregate_merging = context->getConfigRef().getDouble(MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING, 0.9); + config.enable_spill_test = context->getConfigRef().getBool(ENABLE_SPILL_TEST, false); return config; } @@ -136,4 +136,19 @@ GlutenJobSchedulerConfig GlutenJobSchedulerConfig::loadFromContext(const DB::Con config.job_scheduler_max_threads = context->getConfigRef().getUInt64(JOB_SCHEDULER_MAX_THREADS, 10); return config; } -} \ No newline at end of file +MergeTreeCacheConfig MergeTreeCacheConfig::loadFromContext(const DB::ContextPtr & context) +{ + MergeTreeCacheConfig config; + config.enable_data_prefetch = context->getConfigRef().getBool(ENABLE_DATA_PREFETCH, config.enable_data_prefetch); + return config; +} + +WindowConfig WindowConfig::loadFromContext(const DB::ContextPtr & context) +{ + WindowConfig config; + config.aggregate_topk_sample_rows = context->getConfigRef().getUInt64(WINDOW_AGGREGATE_TOPK_SAMPLE_ROWS, 5000); + config.aggregate_topk_high_cardinality_threshold + = context->getConfigRef().getDouble(WINDOW_AGGREGATE_TOPK_HIGH_CARDINALITY_THRESHOLD, 0.6); + return config; +} +} diff --git a/cpp-ch/local-engine/Common/GlutenConfig.h b/cpp-ch/local-engine/Common/GlutenConfig.h index 82402eaafa47..07952383fa10 100644 --- a/cpp-ch/local-engine/Common/GlutenConfig.h +++ b/cpp-ch/local-engine/Common/GlutenConfig.h @@ -19,6 +19,7 @@ #include #include +#include namespace Poco::Util { @@ -33,8 +34,9 @@ namespace local_engine struct SparkConfigs { + using ConfigMap = google::protobuf::Map; static void updateConfig(const DB::ContextMutablePtr &, std::string_view); - static std::map load(std::string_view plan, bool processStart = false); + static void update(std::string_view plan, const std::function & callback, bool processStart = false); }; struct MemoryConfig @@ -54,15 +56,20 @@ struct GraceMergingAggregateConfig { inline static const String MAX_GRACE_AGGREGATE_MERGING_BUCKETS = "max_grace_aggregate_merging_buckets"; inline static const String THROW_ON_OVERFLOW_GRACE_AGGREGATE_MERGING_BUCKETS = "throw_on_overflow_grace_aggregate_merging_buckets"; - inline static const String AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS = "aggregated_keys_before_extend_grace_aggregate_merging_buckets"; - inline static const String MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; - inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING = "max_allowed_memory_usage_ratio_for_aggregate_merging"; + inline static const String AGGREGATED_KEYS_BEFORE_EXTEND_GRACE_AGGREGATE_MERGING_BUCKETS + = "aggregated_keys_before_extend_grace_aggregate_merging_buckets"; + inline static const String MAX_PENDING_FLUSH_BLOCKS_PER_GRACE_AGGREGATE_MERGING_BUCKET + = "max_pending_flush_blocks_per_grace_aggregate_merging_bucket"; + inline static const String MAX_ALLOWED_MEMORY_USAGE_RATIO_FOR_AGGREGATE_MERGING + = "max_allowed_memory_usage_ratio_for_aggregate_merging"; + inline static const String ENABLE_SPILL_TEST = "enable_grace_aggregate_spill_test"; size_t max_grace_aggregate_merging_buckets = 32; bool throw_on_overflow_grace_aggregate_merging_buckets = false; size_t aggregated_keys_before_extend_grace_aggregate_merging_buckets = 8192; size_t max_pending_flush_blocks_per_grace_aggregate_merging_bucket = 1_MiB; double max_allowed_memory_usage_ratio_for_aggregate_merging = 0.9; + bool enable_spill_test = false; static GraceMergingAggregateConfig loadFromContext(const DB::ContextPtr & context); }; @@ -71,7 +78,8 @@ struct StreamingAggregateConfig { inline static const String AGGREGATED_KEYS_BEFORE_STREAMING_AGGREGATING_EVICT = "aggregated_keys_before_streaming_aggregating_evict"; inline static const String MAX_MEMORY_USAGE_RATIO_FOR_STREAMING_AGGREGATING = "max_memory_usage_ratio_for_streaming_aggregating"; - inline static const String HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING = "high_cardinality_threshold_for_streaming_aggregating"; + inline static const String HIGH_CARDINALITY_THRESHOLD_FOR_STREAMING_AGGREGATING + = "high_cardinality_threshold_for_streaming_aggregating"; inline static const String ENABLE_STREAMING_AGGREGATING = "enable_streaming_aggregating"; size_t aggregated_keys_before_streaming_aggregating_evict = 1024; @@ -142,4 +150,29 @@ struct GlutenJobSchedulerConfig static GlutenJobSchedulerConfig loadFromContext(const DB::ContextPtr & context); }; + +struct MergeTreeCacheConfig +{ + inline static const String ENABLE_DATA_PREFETCH = "enable_data_prefetch"; + + bool enable_data_prefetch = true; + + static MergeTreeCacheConfig loadFromContext(const DB::ContextPtr & context); +}; + +struct WindowConfig +{ +public: + inline static const String WINDOW_AGGREGATE_TOPK_SAMPLE_ROWS = "window.aggregate_topk_sample_rows"; + inline static const String WINDOW_AGGREGATE_TOPK_HIGH_CARDINALITY_THRESHOLD = "window.aggregate_topk_high_cardinality_threshold"; + size_t aggregate_topk_sample_rows = 5000; + double aggregate_topk_high_cardinality_threshold = 0.6; + static WindowConfig loadFromContext(const DB::ContextPtr & context); +}; + +namespace PathConfig +{ +inline constexpr const char * USE_CURRENT_DIRECTORY_AS_TMP = "use_current_directory_as_tmp"; +inline constexpr const char * DEFAULT_TEMP_FILE_PATH = "/tmp/libch"; +}; } diff --git a/cpp-ch/local-engine/Common/GlutenSettings.cpp b/cpp-ch/local-engine/Common/GlutenSettings.cpp index ceac59ae1fc3..ab015b56362d 100644 --- a/cpp-ch/local-engine/Common/GlutenSettings.cpp +++ b/cpp-ch/local-engine/Common/GlutenSettings.cpp @@ -41,8 +41,12 @@ bool settingsEqual(const DB::Settings & settings, std::string_view name, const s } void updateSettings(const DB::ContextMutablePtr & context, std::string_view plan) { - std::map spark_conf_map = SparkConfigs::load(plan); - for (const auto & [key, value] : spark_conf_map) - context->setSetting(key, value); + SparkConfigs::update( + plan, + [&](const SparkConfigs::ConfigMap & config_map) + { + for (const auto & [key, value] : config_map) + context->setSetting(key, value); + }); } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Common/GlutenSettings.h b/cpp-ch/local-engine/Common/GlutenSettings.h index 74bb0bb23d50..99c01368b42b 100644 --- a/cpp-ch/local-engine/Common/GlutenSettings.h +++ b/cpp-ch/local-engine/Common/GlutenSettings.h @@ -27,41 +27,41 @@ namespace local_engine #define SKIP_ALIAS(ALIAS_NAME) -#define DECLARE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) TYPE NAME{DEFAULT}; +#define DECLARE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; #define GLUTEN_SETTING_STATIC_MEMBER_(NAME) s_##NAME##_ -#define INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ - static constexpr std::string_view GLUTEN_SETTING_STATIC_MEMBER_(NAME) = "g." #UNIQ "." #NAME; +#define INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ + static constexpr std::string_view GLUTEN_SETTING_STATIC_MEMBER_(NAME) = "gluten." #NAME; #define DECLARE_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ struct SETTINGS_CLASS_NAME \ { \ - LIST_OF_SETTINGS_MACRO(DECLARE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(DECLARE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS) \ static SETTINGS_CLASS_NAME get(const DB::ContextPtr & context); \ void set(const DB::ContextMutablePtr & context) const; \ \ private: \ - LIST_OF_SETTINGS_MACRO(INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS, __COUNTER__) \ + LIST_OF_SETTINGS_MACRO(INITIALIZE_GLUTEN_SETTING_STATIC_MEMBER_, SKIP_ALIAS) \ }; -#define IMPLEMENT_GLUTEN_GET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) \ +#define IMPLEMENT_GLUTEN_GET_(TYPE, NAME, DEFAULT, DESCRIPTION) \ if (DB::Field field_##NAME; settings.tryGet(GLUTEN_SETTING_STATIC_MEMBER_(NAME), field_##NAME)) \ result.NAME = field_##NAME.safeGet(); -#define IMPLEMENT_GLUTEN_SET_(TYPE, NAME, DEFAULT, DESCRIPTION, UNIQ) context->setSetting(GLUTEN_SETTING_STATIC_MEMBER_(NAME), NAME); +#define IMPLEMENT_GLUTEN_SET_(TYPE, NAME, DEFAULT, DESCRIPTION) context->setSetting(GLUTEN_SETTING_STATIC_MEMBER_(NAME), NAME); #define IMPLEMENT_GLUTEN_SETTINGS(SETTINGS_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ SETTINGS_CLASS_NAME SETTINGS_CLASS_NAME::get(const DB::ContextPtr & context) \ { \ SETTINGS_CLASS_NAME result; \ const DB::Settings & settings = context->getSettingsRef(); \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_GET_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_GET_, SKIP_ALIAS) \ return result; \ } \ void SETTINGS_CLASS_NAME::SETTINGS_CLASS_NAME::set(const DB::ContextMutablePtr & context) const \ { \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_SET_, SKIP_ALIAS, _) \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_GLUTEN_SET_, SKIP_ALIAS) \ } // workaround for tryGetString diff --git a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp index 44c43fcb65aa..712d8ddcf5cc 100644 --- a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp +++ b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp @@ -104,7 +104,7 @@ static void writeSignalIDtoSignalPipe(int sig) char buf[signal_pipe_buf_size]; WriteBufferFromFileDescriptor out(writeFD(), signal_pipe_buf_size, buf); writeBinary(sig, out); - out.next(); + out.finalize(); errno = saved_errno; } @@ -251,9 +251,7 @@ class SignalListener : public Poco::Runnable query = thread_ptr->getQueryForLog(); if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) - { CurrentThread::attachInternalTextLogsQueue(logs_queue, LogsLevel::trace); - } } std::string signal_description = "Unknown signal"; diff --git a/cpp-ch/local-engine/Common/GlutenStringUtils.h b/cpp-ch/local-engine/Common/GlutenStringUtils.h index 0d980f228f18..0de185d83cc6 100644 --- a/cpp-ch/local-engine/Common/GlutenStringUtils.h +++ b/cpp-ch/local-engine/Common/GlutenStringUtils.h @@ -21,9 +21,6 @@ namespace local_engine { -using PartitionValue = std::pair; -using PartitionValues = std::vector; - class GlutenStringUtils { public: diff --git a/cpp-ch/local-engine/Common/QueryContext.cpp b/cpp-ch/local-engine/Common/QueryContext.cpp index 4992ba0a455c..eca9ad5b18de 100644 --- a/cpp-ch/local-engine/Common/QueryContext.cpp +++ b/cpp-ch/local-engine/Common/QueryContext.cpp @@ -44,6 +44,7 @@ struct QueryContext::Data std::shared_ptr thread_status; std::shared_ptr thread_group; ContextMutablePtr query_context; + String task_id; static DB::ContextMutablePtr global_context; static SharedContextHolder shared_context; @@ -83,11 +84,12 @@ DB::ContextPtr QueryContext::globalContext() return Data::global_context; } -int64_t QueryContext::initializeQuery() +int64_t QueryContext::initializeQuery(const String & task_id) { std::shared_ptr query_context = std::make_shared(); query_context->query_context = Context::createCopy(globalContext()); query_context->query_context->makeQueryContext(); + query_context->task_id = task_id; // empty input will trigger random query id to be set // FileCache will check if query id is set to decide whether to skip cache or not @@ -95,7 +97,7 @@ int64_t QueryContext::initializeQuery() // // Notice: // this generated random query id a qualified global queryid for the spark query - query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4())); + query_context->query_context->setCurrentQueryId(toString(UUIDHelpers::generateV4()) + "_" + task_id); auto config = MemoryConfig::loadFromContext(query_context->query_context); query_context->thread_status = std::make_shared(false); query_context->thread_group = std::make_shared(query_context->query_context); @@ -124,14 +126,24 @@ std::shared_ptr QueryContext::currentThreadGroup() throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread group not found."); } -void QueryContext::logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const +String QueryContext::currentTaskIdOrEmpty() +{ + if (auto thread_group = CurrentThread::getGroup()) + { + const int64_t id = reinterpret_cast(thread_group.get()); + return query_map_.get(id)->task_id; + } + return ""; +} + +void QueryContext::logCurrentPerformanceCounters(ProfileEvents::Counters & counters, const String & task_id) const { if (!CurrentThread::getGroup()) return; if (logger_->information()) { std::ostringstream msg; - msg << "\n---------------------Task Performance Counters-----------------------------\n"; + msg << "\n---------------------Task Performance Counters(" << task_id << ")-----------------------------\n"; for (ProfileEvents::Event event = ProfileEvents::Event(0); event < counters.num_counters; event++) { const auto * name = ProfileEvents::getName(event); @@ -167,7 +179,7 @@ void QueryContext::finalizeQuery(int64_t id) if (currentThreadGroupMemoryUsage() > 2_MiB) LOG_WARNING(logger_, "{} bytes memory didn't release, There may be a memory leak!", currentThreadGroupMemoryUsage()); - logCurrentPerformanceCounters(context->thread_group->performance_counters); + logCurrentPerformanceCounters(context->thread_group->performance_counters, context->task_id); context->thread_status->detachFromGroup(); context->thread_group.reset(); context->thread_status.reset(); diff --git a/cpp-ch/local-engine/Common/QueryContext.h b/cpp-ch/local-engine/Common/QueryContext.h index 821144f5fcb6..6ced1f632095 100644 --- a/cpp-ch/local-engine/Common/QueryContext.h +++ b/cpp-ch/local-engine/Common/QueryContext.h @@ -40,10 +40,11 @@ class QueryContext static QueryContext instance; return instance; } - int64_t initializeQuery(); + int64_t initializeQuery(const String & task_id); DB::ContextMutablePtr currentQueryContext(); + String currentTaskIdOrEmpty(); static std::shared_ptr currentThreadGroup(); - void logCurrentPerformanceCounters(ProfileEvents::Counters & counters) const; + void logCurrentPerformanceCounters(ProfileEvents::Counters & counters, const String & task_id) const; size_t currentPeakMemory(int64_t id); void finalizeQuery(int64_t id); diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp index 82afeb85e2b8..148a43580b96 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.cpp @@ -28,11 +28,41 @@ bool isMetaDataFile(const std::string & path) return !path.ends_with("bin"); } +TemporaryWriteBufferWrapper::TemporaryWriteBufferWrapper( + const String & file_name_, const std::shared_ptr & data_buffer_) + : WriteBufferFromFileBase(data_buffer_->buffer().size(), data_buffer_->buffer().begin(), 0) + , file_name(file_name_) + , data_buffer(data_buffer_) +{ +} +void TemporaryWriteBufferWrapper::preFinalize() +{ + next(); +} + +void TemporaryWriteBufferWrapper::finalizeImpl() +{ + next(); + data_buffer->finalizeImpl(); +} + +void TemporaryWriteBufferWrapper::cancelImpl() noexcept +{ + data_buffer->cancelImpl(); +} + +void TemporaryWriteBufferWrapper::nextImpl() +{ + data_buffer->position() = position(); + data_buffer->next(); + BufferBase::set(data_buffer->buffer().begin(), data_buffer->buffer().size(), data_buffer->offset()); +} + void CompactObjectStorageDiskTransaction::commit() { auto metadata_tx = disk.getMetadataStorage()->createTransaction(); - std::filesystem::path data_path = std::filesystem::path(prefix_path) / "data.bin"; - std::filesystem::path meta_path = std::filesystem::path(prefix_path) / "meta.bin"; + std::filesystem::path data_path = std::filesystem::path(prefix_path) / PART_DATA_FILE_NAME; + std::filesystem::path meta_path = std::filesystem::path(prefix_path) / PART_META_FILE_NAME; auto object_storage = disk.getObjectStorage(); auto data_key = object_storage->generateObjectKeyForPath(data_path, std::nullopt); @@ -52,9 +82,9 @@ void CompactObjectStorageDiskTransaction::commit() [&](auto & item) { DB::DiskObjectStorageMetadata metadata(object_storage->getCommonKeyPrefix(), item.first); - DB::ReadBufferFromFilePRead read(item.second->getAbsolutePath()); + auto read = item.second->read(); int file_size = 0; - while (int count = read.readBig(buffer.data(), buffer.size())) + while (int count = read->readBig(buffer.data(), buffer.size())) { file_size += count; out.write(buffer.data(), count); @@ -98,12 +128,13 @@ std::unique_ptr CompactObjectStorageDiskTransaction "Don't support write file in different dirs, path {}, prefix path: {}", path, prefix_path); - auto tmp = std::make_shared(tmp_data); + auto tmp = std::make_shared(tmp_data.get()); files.emplace_back(path, tmp); auto tx = disk.getMetadataStorage()->createTransaction(); tx->createDirectoryRecursive(std::filesystem::path(path).parent_path()); tx->createEmptyMetadataFile(path); tx->commit(); - return std::make_unique(tmp->getAbsolutePath(), buf_size); + + return std::make_unique(path, tmp); } } \ No newline at end of file diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h index e15c362f304a..0f95ae01ec2b 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h +++ b/cpp-ch/local-engine/Disks/ObjectStorages/CompactObjectStorageDiskTransaction.h @@ -32,12 +32,41 @@ extern const int NOT_IMPLEMENTED; namespace local_engine { +class TemporaryWriteBufferWrapper : public DB::WriteBufferFromFileBase +{ +public: + TemporaryWriteBufferWrapper(const String & file_name_, const std::shared_ptr & data_buffer_); + + void sync() override { data_buffer->nextImpl(); } + + void preFinalize() override; + +protected: + void finalizeImpl() override; + void cancelImpl() noexcept override; + +private: + void nextImpl() override; + +public: + std::string getFileName() const override + { + return file_name; + } + +private: + String file_name; + std::shared_ptr data_buffer; +}; + class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { public: - explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::DiskPtr tmp_) + static inline const String PART_DATA_FILE_NAME = "part_data.gluten"; + static inline const String PART_META_FILE_NAME = "part_meta.gluten"; + + explicit CompactObjectStorageDiskTransaction(DB::IDisk & disk_, const DB::TemporaryDataOnDiskScopePtr tmp_) : disk(disk_), tmp_data(tmp_) { - chassert(!tmp_->isRemote()); } void commit() override; @@ -167,8 +196,8 @@ class CompactObjectStorageDiskTransaction: public DB::IDiskTransaction { private: DB::IDisk & disk; - DB::DiskPtr tmp_data; - std::vector>> files; + DB::TemporaryDataOnDiskScopePtr tmp_data; + std::vector>> files; String prefix_path = ""; }; } diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp index bd005132b9b9..fed23d7eef61 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskHDFS.cpp @@ -29,7 +29,7 @@ using namespace DB; DiskTransactionPtr GlutenDiskHDFS::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()); } void GlutenDiskHDFS::createDirectory(const String & path) diff --git a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp index b2a6bb523d3a..a180ebd7eaf0 100644 --- a/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp +++ b/cpp-ch/local-engine/Disks/ObjectStorages/GlutenDiskS3.cpp @@ -31,7 +31,7 @@ namespace local_engine DB::DiskTransactionPtr GlutenDiskS3::createTransaction() { - return std::make_shared(*this, QueryContext::globalContext()->getTempDataOnDisk()->getVolume()->getDisk()); + return std::make_shared(*this, QueryContext::globalContext()->getSharedTempDataOnDisk()); } std::unique_ptr GlutenDiskS3::readFile( diff --git a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h b/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h deleted file mode 100644 index 6930c1d75b79..000000000000 --- a/cpp-ch/local-engine/Functions/FunctionGreatestLeast.h +++ /dev/null @@ -1,77 +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. - */ -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} -} -namespace local_engine -{ -template -class FunctionGreatestestLeast : public DB::FunctionLeastGreatestGeneric -{ -public: - bool useDefaultImplementationForNulls() const override { return false; } - virtual String getName() const = 0; - -private: - DB::DataTypePtr getReturnTypeImpl(const DB::DataTypes & types) const override - { - if (types.empty()) - throw DB::Exception(DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} cannot be called without arguments", getName()); - return makeNullable(getLeastSupertype(types)); - } - - DB::ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DB::DataTypePtr & result_type, size_t input_rows_count) const override - { - size_t num_arguments = arguments.size(); - DB::Columns converted_columns(num_arguments); - for (size_t arg = 0; arg < num_arguments; ++arg) - converted_columns[arg] = castColumn(arguments[arg], result_type)->convertToFullColumnIfConst(); - auto result_column = result_type->createColumn(); - result_column->reserve(input_rows_count); - for (size_t row_num = 0; row_num < input_rows_count; ++row_num) - { - size_t best_arg = 0; - for (size_t arg = 1; arg < num_arguments; ++arg) - { - if constexpr (kind == DB::LeastGreatest::Greatest) - { - auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], -1); - if (cmp_result > 0) - best_arg = arg; - } - else - { - auto cmp_result = converted_columns[arg]->compareAt(row_num, row_num, *converted_columns[best_arg], 1); - if (cmp_result < 0) - best_arg = arg; - } - } - result_column->insertFrom(*converted_columns[best_arg], row_num); - } - return result_column; - } -}; - -} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp index 4c2847d9f92a..bf65b253479b 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArrayJoin.cpp @@ -14,13 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include -#include -#include +#include +#include #include -#include -#include +#include +#include #include +#include +#include +#include +#include using namespace DB; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp b/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp index e43b52823175..ea841632a984 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionArraysOverlap.cpp @@ -14,12 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include +#include #include -#include -#include -#include +#include +#include #include +#include +#include using namespace DB; @@ -92,7 +93,7 @@ class SparkFunctionArraysOverlap : public IFunction { res_data[i] = 1; null_map_data[i] = 0; - break; + break; } } } diff --git a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp index c75d25b6ef80..8b5a7eff65db 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionCheckDecimalOverflow.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp index 830fc0e65287..8b79bae90d0c 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionDecimalBinaryArithmetic.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -26,8 +27,6 @@ #include #include #include -#include -#include #include namespace DB @@ -488,7 +487,7 @@ class SparkFunctionDecimalBinaryArithmetic final : public IFunction right_generic, removeNullable(arguments[2].type).get(), [&](const auto & left, const auto & right, const auto & result) { - return (res = SparkDecimalBinaryOperation::template executeDecimal(arguments, left, right, result)) + return (res = SparkDecimalBinaryOperation::template executeDecimal<>(arguments, left, right, result)) != nullptr; }); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h b/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h deleted file mode 100644 index 1b93e77d9400..000000000000 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.h +++ /dev/null @@ -1,176 +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. - */ - -#include -#include -#include -#include -#include -#include - -using namespace DB; - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; -} -} - -namespace local_engine -{ -struct DivideDecimalsImpl -{ - static constexpr auto name = "sparkDivideDecimal"; - template - static inline Decimal256 - execute(FirstType a, SecondType b, UInt16 scale_a, UInt16 scale_b, UInt16 result_scale) - { - if (a.value == 0 || b.value == 0) - return Decimal256(0); - - Int256 sign_a = a.value < 0 ? -1 : 1; - Int256 sign_b = b.value < 0 ? -1 : 1; - - std::vector a_digits = DecimalOpHelpers::toDigits(a.value * sign_a); - - while (scale_a < scale_b + result_scale) - { - a_digits.push_back(0); - ++scale_a; - } - - while (scale_a > scale_b + result_scale && !a_digits.empty()) - { - a_digits.pop_back(); - --scale_a; - } - - if (a_digits.empty()) - return Decimal256(0); - - std::vector divided = DecimalOpHelpers::divide(a_digits, b.value * sign_b); - - if (divided.size() > DecimalUtils::max_precision) - throw DB::Exception(ErrorCodes::DECIMAL_OVERFLOW, "Numeric overflow: result bigger that Decimal256"); - return Decimal256(sign_a * sign_b * DecimalOpHelpers::fromDigits(divided)); - } -}; - -template -class SparkFunctionDecimalDivide : public FunctionsDecimalArithmetics -{ -public: - static constexpr auto name = Transform::name; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionDecimalDivide() = default; - ~SparkFunctionDecimalDivide() override = default; - String getName() const override { return name; } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - return makeNullable(FunctionsDecimalArithmetics::getReturnTypeImpl(arguments)); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows) const override - { - if (arguments.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} must have 2 arugments.", name); - - ColumnPtr res_col = nullptr; - MutableColumnPtr null_map_col = ColumnUInt8::create(input_rows, 0); - auto getNonNullableColumn = [&](const ColumnPtr & col) -> const ColumnPtr - { - if (col->isNullable()) - { - auto * nullable_col = checkAndGetColumn(col.get()); - return nullable_col->getNestedColumnPtr(); - } - else - return col; - }; - - ColumnWithTypeAndName new_arg0 {getNonNullableColumn(arguments[0].column), removeNullable(arguments[0].type), arguments[0].name}; - ColumnWithTypeAndName new_arg1 {getNonNullableColumn(arguments[1].column), removeNullable(arguments[1].type), arguments[0].name}; - ColumnsWithTypeAndName new_args {new_arg0, new_arg1}; - bool arg_type_valid = true; - - if (isDecimal(new_arg1.type)) - { - using Types = TypeList; - arg_type_valid = castTypeToEither(Types{}, new_arg1.type.get(), [&](const auto & right_) - { - using R = typename std::decay_t::FieldType; - const ColumnDecimal * const_col_right = checkAndGetColumnConstData>(new_arg1.column.get()); - if (const_col_right && const_col_right->getElement(0).value == 0) - { - null_map_col = ColumnUInt8::create(input_rows, 1); - res_col = ColumnDecimal::create(input_rows, 0); - } - else - res_col = FunctionsDecimalArithmetics::executeImpl(new_args, removeNullable(result_type), input_rows); - - if (!const_col_right) - { - const ColumnDecimal * col_right = assert_cast *>(new_arg1.column.get()); - PaddedPODArray & null_map = assert_cast*>(null_map_col.get())->getData(); - for (size_t i = 0; i < col_right->size(); ++i) - null_map[i] = (col_right->getElement(i).value == 0 || arguments[1].column->isNullAt(i)); - } - return true; - }); - } - else if (isNumber(new_arg1.type)) - { - using Types = TypeList; - arg_type_valid = castTypeToEither(Types{}, new_arg1.type.get(), [&](const auto & right_) - { - using R = typename std::decay_t::FieldType; - const ColumnVector * const_col_right = checkAndGetColumnConstData>(new_arg1.column.get()); - if (const_col_right && const_col_right->getElement(0) == 0) - { - null_map_col = ColumnUInt8::create(input_rows, 1); - res_col = ColumnDecimal::create(input_rows, 0); - } - else - res_col = FunctionsDecimalArithmetics::executeImpl(new_args, removeNullable(result_type), input_rows); - - if (!const_col_right) - { - const ColumnVector * col_right = assert_cast *>(new_arg1.column.get()); - PaddedPODArray & null_map = assert_cast*>(null_map_col.get())->getData(); - for (size_t i = 0; i < col_right->size(); ++i) - null_map[i] = (col_right->getElement(i) == 0 || arguments[1].column->isNullAt(i)); - } - return true; - }); - } - else - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s arguments type must be numbeic", name); - - if (!arg_type_valid) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s arguments type is not valid.", name); - - return ColumnNullable::create(res_col, std::move(null_map_col)); - } - -}; -} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h index 4f655c615b9f..c9058a564450 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionFloor.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionFloor.h @@ -23,6 +23,10 @@ #include #include +#if USE_MULTITARGET_CODE +#include +#endif + using namespace DB; namespace local_engine diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp index 795e2b0be329..f136f587c539 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp +++ b/cpp-ch/local-engine/Functions/SparkFunctionMakeDecimal.cpp @@ -15,12 +15,12 @@ * limitations under the License. */ #include +#include #include #include #include #include -#include "SparkFunctionCheckDecimalOverflow.h" - +#include namespace DB { diff --git a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h index 444e4542c807..5541245244a7 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionMapToString.h @@ -14,17 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - +#pragma once #include +#include #include #include -#include #include #include -#include #include +#include #include #include +#include #include namespace DB @@ -76,7 +77,10 @@ class SparkFunctionMapToString : public DB::IFunction arguments[1].type->getName(), arguments[2].type->getName()); } - return makeNullable(std::make_shared()); + if (arguments[0].type->isNullable()) + return makeNullable(std::make_shared()); + else + return std::make_shared(); } ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows*/) const override @@ -89,7 +93,8 @@ class SparkFunctionMapToString : public DB::IFunction } const auto & col_with_type_and_name = columnGetNested(arguments[0]); - const IColumn & col_from = *col_with_type_and_name.column; + const IColumn & column = *col_with_type_and_name.column; + const IColumn & col_from = column.isConst() ? reinterpret_cast(column).getDataColumn() : column; size_t size = col_from.size(); auto col_to = removeNullable(result_type)->createColumn(); diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 432595e09140..2dfd25772ee7 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -111,7 +111,14 @@ class BaseFloatRoundingHalfUpComputation template static VectorType apply(VectorType val) { - return roundWithMode(val, mode); + if constexpr (std::is_same_v) + { + return std::roundf(val); + } + else + { + return std::round(val); + } } static VectorType prepare(size_t scale) @@ -120,6 +127,31 @@ class BaseFloatRoundingHalfUpComputation } }; +template <> +class BaseFloatRoundingHalfUpComputation +{ +public: + using ScalarType = BFloat16; + using VectorType = BFloat16; + static const size_t data_count = 1; + + static VectorType load(const ScalarType * in) { return *in; } + static VectorType load1(const ScalarType in) { return in; } + static VectorType store(ScalarType * out, ScalarType val) { return *out = val;} + static VectorType multiply(VectorType val, VectorType scale) { return val * scale; } + static VectorType divide(VectorType val, VectorType scale) { return val / scale; } + template + static VectorType apply(VectorType val) + { + return BFloat16(std::roundf(static_cast(val))); + } + + static VectorType prepare(size_t scale) + { + return load1(BFloat16(static_cast(scale))); + } +}; + /** Implementation of low-level round-off functions for floating-point values. */ @@ -160,7 +192,7 @@ struct FloatRoundingHalfUpImpl template ? Vectorize::No : Vectorize::Yes #else Vectorize::No #endif @@ -212,7 +244,7 @@ struct DispatcherRoundingHalfUp { template using FunctionRoundingImpl = std::conditional_t< - std::is_floating_point_v, + std::is_floating_point_v || std::is_same_v, FloatRoundingHalfUpImpl, IntegerRoundingImpl>; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp new file mode 100644 index 000000000000..1868c40c0f31 --- /dev/null +++ b/cpp-ch/local-engine/Functions/SparkFunctionSplitByRegexp.cpp @@ -0,0 +1,239 @@ +/* + * 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. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; +} + + +/** Functions that split strings into an array of strings or vice versa. + * + * splitByRegexp(regexp, s[, max_substrings]) + */ +namespace +{ + +using Pos = const char *; + +class SparkSplitByRegexpImpl +{ +private: + Regexps::RegexpPtr re; + OptimizedRegularExpression::MatchVec matches; + + Pos pos; + Pos end; + + std::optional max_splits; + size_t splits; + bool max_substrings_includes_remaining_string; + +public: + static constexpr auto name = "splitByRegexpSpark"; + + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } + + static ColumnNumbers getArgumentsThatAreAlwaysConstant() { return {0, 2}; } + + static void checkArguments(const IFunction & func, const ColumnsWithTypeAndName & arguments) + { + checkArgumentsWithSeparatorAndOptionalMaxSubstrings(func, arguments); + } + + static constexpr auto strings_argument_position = 1uz; + + void init(const ColumnsWithTypeAndName & arguments, bool max_substrings_includes_remaining_string_) + { + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + + if (!col) + throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}. " + "Must be constant string.", arguments[0].column->getName(), name); + + if (!col->getValue().empty()) + re = std::make_shared(Regexps::createRegexp(col->getValue())); + + max_substrings_includes_remaining_string = max_substrings_includes_remaining_string_; + max_splits = extractMaxSplits(arguments, 2); + } + + /// Called for each next string. + void set(Pos pos_, Pos end_) + { + pos = pos_; + end = end_; + splits = 0; + } + + /// Get the next token, if any, or return false. + bool get(Pos & token_begin, Pos & token_end) + { + if (!re) + { + if (pos == end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = end; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + ++pos; + token_end = pos; + ++splits; + } + else + { + if (!pos || pos > end) + return false; + + token_begin = pos; + + if (max_splits) + { + if (max_substrings_includes_remaining_string) + { + if (splits == *max_splits - 1) + { + token_end = end; + pos = nullptr; + return true; + } + } + else + if (splits == *max_splits) + return false; + } + + auto res = re->match(pos, end - pos, matches); + if (!res) + { + token_end = end; + pos = end + 1; + } + else if (!matches[0].length) + { + /// If match part is empty, increment position to avoid infinite loop. + token_end = (pos == end ? end : pos + 1); + ++pos; + ++splits; + } + else + { + token_end = pos + matches[0].offset; + pos = token_end + matches[0].length; + ++splits; + } + } + + return true; + } +}; + +using SparkFunctionSplitByRegexp = FunctionTokens; + +/// Fallback splitByRegexp to splitByChar when its 1st argument is a trivial char for better performance +class SparkSplitByRegexpOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "splitByRegexpSpark"; + static FunctionOverloadResolverPtr create(ContextPtr context) { return std::make_unique(context); } + + explicit SparkSplitByRegexpOverloadResolver(ContextPtr context_) + : context(context_) + , split_by_regexp(SparkFunctionSplitByRegexp::create(context)) {} + + String getName() const override { return name; } + size_t getNumberOfArguments() const override { return SparkSplitByRegexpImpl::getNumberOfArguments(); } + bool isVariadic() const override { return SparkSplitByRegexpImpl::isVariadic(); } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + if (patternIsTrivialChar(arguments)) + return FunctionFactory::instance().getImpl("splitByChar", context)->build(arguments); + return std::make_unique( + split_by_regexp, collections::map(arguments, [](const auto & elem) { return elem.type; }), return_type); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + return split_by_regexp->getReturnTypeImpl(arguments); + } + +private: + bool patternIsTrivialChar(const ColumnsWithTypeAndName & arguments) const + { + if (!arguments[0].column.get()) + return false; + const ColumnConst * col = checkAndGetColumnConstStringOrFixedString(arguments[0].column.get()); + if (!col) + return false; + + String pattern = col->getValue(); + if (pattern.size() == 1) + { + OptimizedRegularExpression re = Regexps::createRegexp(pattern); + + std::string required_substring; + bool is_trivial; + bool required_substring_is_prefix; + re.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + return is_trivial && required_substring == pattern; + } + return false; + } + + ContextPtr context; + FunctionPtr split_by_regexp; +}; +} + +REGISTER_FUNCTION(SparkSplitByRegexp) +{ + factory.registerFunction(); +} + +} diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp new file mode 100644 index 000000000000..72721ce85921 --- /dev/null +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.cpp @@ -0,0 +1,270 @@ +/* + * 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. + */ + +#include "AdvancedExpandStep.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace local_engine +{ + +static DB::ITransformingStep::Traits getTraits() +{ + return DB::ITransformingStep::Traits{ + { + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + }}; +} + +AdvancedExpandStep::AdvancedExpandStep( + DB::ContextPtr context_, + const DB::Block & input_header_, + size_t grouping_keys_, + const DB::AggregateDescriptions & aggregate_descriptions_, + const ExpandField & project_set_exprs_) + : DB::ITransformingStep(input_header_, buildOutputHeader(input_header_, project_set_exprs_), getTraits()) + , context(context_) + , grouping_keys(grouping_keys_) + , aggregate_descriptions(aggregate_descriptions_) + , project_set_exprs(project_set_exprs_) +{ +} + +DB::Block AdvancedExpandStep::buildOutputHeader(const DB::Block &, const ExpandField & project_set_exprs_) +{ + DB::ColumnsWithTypeAndName cols; + const auto & types = project_set_exprs_.getTypes(); + const auto & names = project_set_exprs_.getNames(); + + chassert(names.size() == types.size()); + + for (size_t i = 0; i < project_set_exprs_.getExpandCols(); ++i) + cols.emplace_back(DB::ColumnWithTypeAndName(types[i], names[i])); + + return DB::Block(std::move(cols)); +} + +void AdvancedExpandStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & pipeline_settings) +{ + const auto & settings = context->getSettingsRef(); + DB::Names aggregate_grouping_keys; + for (size_t i = 0; i < output_header->columns(); ++i) + { + const auto & col = output_header->getByPosition(i); + if (typeid_cast(col.column.get())) + break; + aggregate_grouping_keys.push_back(col.name); + } + // partial to partial aggregate + auto params = AggregatorParamsHelper::buildParams( + context, aggregate_grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::PARTIAL_TO_PARTIAL); + + auto input_header = input_headers.front(); + auto build_transform = [&](DB::OutputPortRawPtrs outputs) + { + DB::Processors new_processors; + for (auto & output : outputs) + { + auto expand_processor + = std::make_shared(input_header, *output_header, grouping_keys, project_set_exprs); + DB::connect(*output, expand_processor->getInputs().front()); + new_processors.push_back(expand_processor); + + auto expand_output_header = expand_processor->getOutputs().front().getHeader(); + + auto transform_params = std::make_shared(expand_output_header, params, false); + auto aggregate_processor = std::make_shared(context, expand_output_header, transform_params); + DB::connect(expand_processor->getOutputs().back(), aggregate_processor->getInputs().front()); + new_processors.push_back(aggregate_processor); + auto aggregate_output_header = aggregate_processor->getOutputs().front().getHeader(); + + auto resize_processor = std::make_shared(expand_output_header, 2, 1); + DB::connect(aggregate_processor->getOutputs().front(), resize_processor->getInputs().front()); + DB::connect(expand_processor->getOutputs().front(), resize_processor->getInputs().back()); + new_processors.push_back(resize_processor); + } + return new_processors; + }; + pipeline.transform(build_transform); +} + +void AdvancedExpandStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} + +void AdvancedExpandStep::updateOutputHeader() +{ + output_header = buildOutputHeader(input_headers.front(), project_set_exprs); +} + +/// It has two output ports. The 1st output port is for high cardinality data, the 2nd output port is for +/// low cardinality data. +AdvancedExpandTransform::AdvancedExpandTransform( + const DB::Block & input_header_, const DB::Block & output_header_, size_t grouping_keys_, const ExpandField & project_set_exprs_) + : DB::IProcessor({input_header_}, {output_header_, output_header_}) + , grouping_keys(grouping_keys_) + , project_set_exprs(project_set_exprs_) + , input_header(input_header_) +{ + for (size_t i = 0; i < project_set_exprs.getExpandRows(); ++i) + { + const auto & kinds = project_set_exprs.getKinds()[i]; + size_t n = 0; + for (size_t k = 0; k < grouping_keys; ++k) + if (kinds[k] == EXPAND_FIELD_KIND_SELECTION) + n += 1; + need_to_aggregate.push_back((n != grouping_keys)); + } + + for (auto & port : outputs) + output_ports.push_back(&port); +} + +DB::IProcessor::Status AdvancedExpandTransform::prepare() +{ + auto & input = inputs.front(); + + if (isCancelled() || output_ports[0]->isFinished() || output_ports[1]->isFinished()) + { + input.close(); + output_ports[0]->finish(); + output_ports[1]->finish(); + return Status::Finished; + } + + if (has_output) + { + auto & output_port = *output_ports[need_to_aggregate[expand_expr_iterator - 1]]; + if (output_port.canPush()) + { + output_blocks[need_to_aggregate[expand_expr_iterator - 1]] += 1; + output_rows[need_to_aggregate[expand_expr_iterator - 1]] += output_chunk.getNumRows(); + output_port.push(std::move(output_chunk)); + has_output = false; + auto status = expand_expr_iterator >= project_set_exprs.getExpandRows() ? Status::NeedData : Status::Ready; + return status; + } + else + { + return Status::PortFull; + } + } + + if (!has_input) + { + if (input.isFinished()) + { + LOG_DEBUG( + getLogger("AdvancedExpandTransform"), + "Input rows/blocks={}/{}. output rows/blocks=[{}/{}, {}/{}]", + input_rows, + input_blocks, + output_rows[0], + output_blocks[0], + output_rows[1], + output_blocks[1]); + output_ports[0]->finish(); + output_ports[1]->finish(); + return Status::Finished; + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + input_chunk = input.pull(true); + has_input = true; + expand_expr_iterator = 0; + input_blocks += 1; + input_rows += input_chunk.getNumRows(); + } + + return Status::Ready; +} + +void AdvancedExpandTransform::work() +{ + expandInputChunk(); +} + +void AdvancedExpandTransform::expandInputChunk() +{ + const auto & input_columns = input_chunk.getColumns(); + const auto & types = project_set_exprs.getTypes(); + const auto & kinds = project_set_exprs.getKinds()[expand_expr_iterator]; + const auto & fields = project_set_exprs.getFields()[expand_expr_iterator]; + size_t rows = input_chunk.getNumRows(); + + DB::Columns columns(types.size()); + for (size_t col_i = 0; col_i < types.size(); ++col_i) + { + const auto & type = types[col_i]; + const auto & kind = kinds[col_i]; + const auto & field = fields[col_i]; + + if (kind == EXPAND_FIELD_KIND_SELECTION) + { + auto index = field.safeGet(); + const auto & input_column = input_columns[index]; + + DB::ColumnWithTypeAndName input_arg; + input_arg.column = input_column; + input_arg.type = input_header.getByPosition(index).type; + /// input_column maybe non-Nullable + columns[col_i] = DB::castColumn(input_arg, type); + } + else if (kind == EXPAND_FIELD_KIND_LITERAL) + { + /// Add const column with field value + auto column = type->createColumnConst(rows, field)->convertToFullColumnIfConst(); + columns[col_i] = std::move(column); + } + else + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknown ExpandFieldKind {}", magic_enum::enum_name(kind)); + } + + output_chunk = DB::Chunk(std::move(columns), rows); + has_output = true; + + ++expand_expr_iterator; + has_input = expand_expr_iterator < project_set_exprs.getExpandRows(); +} +} diff --git a/cpp-ch/local-engine/Operator/AdvancedExpandStep.h b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h new file mode 100644 index 000000000000..343a7f5227ee --- /dev/null +++ b/cpp-ch/local-engine/Operator/AdvancedExpandStep.h @@ -0,0 +1,96 @@ +/* + * 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. + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ +// This step is used when we move the expand operator after the partial aggregator. +// To avoid increasing the overhead of shuffle when some of the grouping keys are high cardinality, we add an extra aggregate operator after +// this expand operator and aggregate the low cardinality grouping keys. +class AdvancedExpandStep : public DB::ITransformingStep +{ +public: + explicit AdvancedExpandStep( + DB::ContextPtr context_, + const DB::Block & input_header_, + size_t grouping_keys_, + const DB::AggregateDescriptions & aggregate_descriptions_, + const ExpandField & project_set_exprs_); + ~AdvancedExpandStep() override = default; + + String getName() const override { return "AdvancedExpandStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + + static DB::Block buildOutputHeader(const DB::Block & header, const ExpandField & project_set_exprs_); + +protected: + DB::ContextPtr context; + size_t grouping_keys; + DB::AggregateDescriptions aggregate_descriptions; + ExpandField project_set_exprs; + + void updateOutputHeader() override; +}; + +class AdvancedExpandTransform : public DB::IProcessor +{ +public: + using Status = DB::IProcessor::Status; + /// Need to ensure that the input header is [grouping keys] ++ [aggregation columns] + explicit AdvancedExpandTransform( + const DB::Block & inpput_header_, const DB::Block & output_header_, size_t goruping_keys_, const ExpandField & project_set_exprs_); + ~AdvancedExpandTransform() override = default; + + Status prepare() override; + void work() override; + String getName() const override { return "AdvancedExpandTransform"; } + +private: + size_t grouping_keys = 0; + ExpandField project_set_exprs; + DB::Block input_header; + bool has_input = false; + bool has_output = false; + size_t expand_expr_iterator = 0; + std::vector need_to_aggregate; + + std::vector output_ports; + + DB::Chunk input_chunk; + DB::Chunk output_chunk; + + size_t input_blocks = 0; + size_t input_rows = 0; + std::vector output_blocks = {0, 0}; + std::vector output_rows = {0, 0}; + + void expandInputChunk(); +}; + +} diff --git a/cpp-ch/local-engine/Operator/BranchStep.cpp b/cpp-ch/local-engine/Operator/BranchStep.cpp new file mode 100644 index 000000000000..5e379ae9d4dc --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchStep.cpp @@ -0,0 +1,265 @@ +/* + * 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. + */ + +#include "BranchStep.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ + +class BranchHookSource : public DB::IProcessor +{ +public: + using Status = DB::IProcessor::Status; + BranchHookSource(const DB::Block & header_) : DB::IProcessor({}, {header_}) { inner_inputs.emplace_back(header_, this); } + ~BranchHookSource() override = default; + + String getName() const override { return "BranchHookSource"; } + + Status prepare() override; + void work() override; + void enableInputs() { inputs.swap(inner_inputs); } + +private: + DB::InputPorts inner_inputs; + bool has_output = false; + DB::Chunk output_chunk; + bool has_input = false; + DB::Chunk input_chunk; +}; + +BranchHookSource::Status BranchHookSource::prepare() +{ + auto & output = outputs.front(); + auto & input = inputs.front(); + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + if (has_output) + { + if (output.canPush()) + { + output.push(std::move(output_chunk)); + has_output = false; + } + return Status::PortFull; + } + if (has_input) + return Status::Ready; + if (input.isFinished()) + { + output.finish(); + return Status::Finished; + } + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + input_chunk = input.pull(true); + has_input = true; + return Status::Ready; +} + +void BranchHookSource::work() +{ + if (has_input) + { + output_chunk = std::move(input_chunk); + has_output = true; + has_input = false; + } +} + + +static DB::ITransformingStep::Traits getTraits() +{ + return DB::ITransformingStep::Traits{ + { + .returns_single_stream = true, + .preserves_number_of_streams = false, + .preserves_sorting = false, + }, + { + .preserves_number_of_rows = false, + }}; +} + +class ResizeStep : public DB::ITransformingStep +{ +public: + explicit ResizeStep(const DB::Block & header_, size_t num_streams_) + : DB::ITransformingStep(header_, header_, getTraits()), num_streams(num_streams_) + { + } + ~ResizeStep() override = default; + + String getName() const override { return "UniteBranchesStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings &) override + { + LOG_ERROR(getLogger("ResizeStep"), "xxx num_streams: {}", num_streams); + pipeline.resize(num_streams); + } + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override + { + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); + } + +private: + size_t num_streams; + void updateOutputHeader() override {}; +}; + +DB::QueryPlanPtr BranchStepHelper::createSubPlan(const DB::Block & header, size_t num_streams) +{ + auto source = std::make_unique(DB::Pipe(std::make_shared(header))); + source->setStepDescription("Hook node connected to one branch output"); + auto plan = std::make_unique(); + plan->addStep(std::move(source)); + + if (num_streams > 1) + { + auto resize_step = std::make_unique(plan->getCurrentHeader(), num_streams); + plan->addStep(std::move(resize_step)); + } + return std::move(plan); +} + +StaticBranchStep::StaticBranchStep( + DB::ContextPtr context_, const DB::Block & header_, size_t branches_, size_t sample_rows_, BranchSelector selector_) + : DB::ITransformingStep(header_, header_, getTraits()) + , context(context_) + , header(header_) + , branches(branches_) + , max_sample_rows(sample_rows_) + , selector(selector_) +{ +} + +void StaticBranchStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) +{ + auto build_transform = [&](DB::OutputPortRawPtrs child_outputs) -> DB::Processors + { + DB::Processors new_processors; + for (auto & output : child_outputs) + { + if (!output) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Output port is null"); + auto branch_transform = std::make_shared(header, max_sample_rows, branches, selector); + DB::connect(*output, branch_transform->getInputs().front()); + new_processors.push_back(branch_transform); + } + return new_processors; + }; + pipeline.resize(1); + pipeline.transform(build_transform); +} + +void StaticBranchStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} + +void StaticBranchStep::updateOutputHeader() +{ +} + +UniteBranchesStep::UniteBranchesStep(const DB::Block & header_, std::vector && branch_plans_, size_t num_streams_) + : DB::ITransformingStep(header_, branch_plans_[0]->getCurrentHeader(), getTraits()), header(header_) +{ + branch_plans.swap(branch_plans_); + size_t branches = branch_plans.size(); + num_streams = num_streams_; +} + +void UniteBranchesStep::transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings &) +{ + auto add_transform = [&](DB::OutputPortRawPtrs child_outputs) -> DB::Processors + { + DB::Processors new_processors; + size_t branch_index = 0; + if (child_outputs.size() != branch_plans.size()) + { + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Output port's size({}) is not equal to branches size({})", + child_outputs.size(), + branch_plans.size()); + } + for (auto output : child_outputs) + { + auto & branch_plan = branch_plans[branch_index]; + DB::QueryPlanOptimizationSettings optimization_settings; + DB::BuildQueryPipelineSettings build_settings; + DB::QueryPlanResourceHolder resource_holder; + + auto pipeline_builder = branch_plan->buildQueryPipeline(optimization_settings, build_settings); + auto pipe = DB::QueryPipelineBuilder::getPipe(std::move(*pipeline_builder), resource_holder); + DB::ProcessorPtr source_node = nullptr; + auto processors = DB::Pipe::detachProcessors(std::move(pipe)); + for (auto processor : processors) + { + if (auto * source = typeid_cast(processor.get())) + { + if (source->getInputs().empty()) + { + if (source_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "There is multi source in branch plan"); + source->enableInputs(); + source_node = processor; + } + } + } + if (!source_node) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Cannot find source node in branch plan"); + if (source_node->getInputs().empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Source node has no input"); + DB::connect(*output, source_node->getInputs().front()); + new_processors.insert(new_processors.end(), processors.begin(), processors.end()); + branch_index++; + } + return new_processors; + }; + pipeline.transform(add_transform); + pipeline.resize(1); + if (num_streams > 1) + pipeline.resize(num_streams); +} + +void UniteBranchesStep::describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const +{ + if (!processors.empty()) + DB::IQueryPlanStep::describePipeline(processors, settings); +} +} diff --git a/cpp-ch/local-engine/Operator/BranchStep.h b/cpp-ch/local-engine/Operator/BranchStep.h new file mode 100644 index 000000000000..fd2203aae80a --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchStep.h @@ -0,0 +1,86 @@ +/* + * 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. + */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace local_engine +{ + +class BranchStepHelper +{ +public: + // Create a new query plan that would be used to build sub branch query plan. + static DB::QueryPlanPtr createSubPlan(const DB::Block & header, size_t num_streams); +}; + +// Use to branch the query plan. +class StaticBranchStep : public DB::ITransformingStep +{ +public: + using BranchSelector = std::function &)>; + explicit StaticBranchStep( + DB::ContextPtr context_, const DB::Block & header, size_t branches, size_t sample_rows, BranchSelector selector); + ~StaticBranchStep() override = default; + + String getName() const override { return "StaticBranchStep"; } + + // This will resize the num_streams to 1. You may need to resize after this. + void transformPipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings & settings) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + +protected: + void updateOutputHeader() override; + +private: + DB::ContextPtr context; + DB::Block header; + size_t max_sample_rows; + size_t branches; + BranchSelector selector; +}; + + +// It should be better to build execution branches on QueryPlan. +class UniteBranchesStep : public DB::ITransformingStep +{ +public: + explicit UniteBranchesStep(const DB::Block & header_, std::vector && branch_plans_, size_t num_streams_); + ~UniteBranchesStep() override = default; + + String getName() const override { return "UniteBranchesStep"; } + + void transformPipeline(DB::QueryPipelineBuilder & pipelines, const DB::BuildQueryPipelineSettings &) override; + void describePipeline(DB::IQueryPlanStep::FormatSettings & settings) const override; + +private: + DB::Block header; + std::vector branch_plans; + size_t num_streams; + + void updateOutputHeader() override { output_header = header; }; +}; + +} diff --git a/cpp-ch/local-engine/Operator/BranchTransform.cpp b/cpp-ch/local-engine/Operator/BranchTransform.cpp new file mode 100644 index 000000000000..f923f4ac4b41 --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchTransform.cpp @@ -0,0 +1,155 @@ + +/* + * 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. + */ +#include "BranchTransform.h" +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + +namespace local_engine +{ +static DB::OutputPorts buildOutputPorts(const DB::Block & header, size_t branches) +{ + DB::OutputPorts output_ports; + for (size_t i = 0; i < branches; ++i) + output_ports.emplace_back(header); + return output_ports; +} +StaticBranchTransform::StaticBranchTransform(const DB::Block & header_, size_t sample_rows_, size_t branches_, BranchSelector selector_) + : DB::IProcessor({header_}, buildOutputPorts(header_, branches_)), max_sample_rows(sample_rows_), selector(selector_) +{ +} + +static bool existFinishedOutput(const DB::OutputPorts & output_ports) +{ + for (const auto & output_port : output_ports) + if (output_port.isFinished()) + return true; + return false; +} + +StaticBranchTransform::Status StaticBranchTransform::prepare() +{ + auto & input = inputs.front(); + if ((selected_output_port && selected_output_port->isFinished()) || (!selected_output_port && existFinishedOutput(outputs))) + { + input.close(); + return Status::Finished; + } + + if (has_output) + { + assert(selected_output_port != nullptr); + if (selected_output_port->canPush()) + { + selected_output_port->push(std::move(output_chunk)); + has_output = false; + } + return Status::PortFull; + } + + if (has_input || (selected_output_port && !sample_chunks.empty())) + { + // to clear the pending chunks + return Status::Ready; + } + + if (input.isFinished()) + { + if (!sample_chunks.empty()) + { + // to clear the pending chunks + return Status::Ready; + } + else + { + if (selected_output_port) + selected_output_port->finish(); + else + for (auto & output_port : outputs) + output_port.finish(); + return Status::Finished; + } + } + + input.setNeeded(); + if (!input.hasData()) + return Status::NeedData; + input_chunk = input.pull(true); + has_input = true; + return Status::Ready; +} + +void StaticBranchTransform::work() +{ + if (selected_output_port) + { + if (!sample_chunks.empty()) + { + assert(!has_input); + has_output = true; + output_chunk.swap(sample_chunks.front()); + sample_chunks.pop_front(); + } + else + { + assert(has_input); + has_input = false; + has_output = true; + output_chunk.swap(input_chunk); + } + } + else if (has_input) + { + sample_rows += input_chunk.getNumRows(); + sample_chunks.emplace_back(std::move(input_chunk)); + if (sample_rows >= max_sample_rows) + setupOutputPort(); + has_input = false; + } + else if (!sample_chunks.empty()) + { + if (!selected_output_port) + setupOutputPort(); + output_chunk.swap(sample_chunks.front()); + sample_chunks.pop_front(); + has_output = true; + } +} + +void StaticBranchTransform::setupOutputPort() +{ + size_t branch_index = selector(sample_chunks); + LOG_DEBUG(getLogger("StaticBranchTransform"), "Select output port: {}", branch_index); + if (branch_index >= outputs.size()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Branch index {} is out of range(0, {})", branch_index, outputs.size()); + auto it = outputs.begin(); + std::advance(it, branch_index); + selected_output_port = &(*it); + // close other output ports + for (auto oit = outputs.begin(); oit != outputs.end(); ++oit) + if (oit != it) + oit->finish(); +} +} // namespace local_engine diff --git a/cpp-ch/local-engine/Operator/BranchTransform.h b/cpp-ch/local-engine/Operator/BranchTransform.h new file mode 100644 index 000000000000..f5284b5ae968 --- /dev/null +++ b/cpp-ch/local-engine/Operator/BranchTransform.h @@ -0,0 +1,56 @@ +/* + * 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. + */ + +#pragma once +#include +#include +#include + +namespace local_engine +{ + +// This is designed for adaptive execution. It has multiple outputs, each indicates for a execution branches. +// It accepts a branch selector, this selector will analysis the input data, and select one of the output port +// as the final only output port. Other output ports will be closed. +// The output port cannot be changed once it's selected. +class StaticBranchTransform : public DB::IProcessor +{ +public: + using BranchSelector = std::function &)>; + using Status = DB::IProcessor::Status; + StaticBranchTransform(const DB::Block & header_, size_t sample_rows_, size_t branches_, BranchSelector selector_); + + String getName() const override { return "StaticBranchTransform"; } + + Status prepare() override; + void work() override; + +private: + size_t max_sample_rows; + BranchSelector selector; + DB::OutputPort * selected_output_port = nullptr; + std::list sample_chunks; + size_t sample_rows = 0; + bool has_input = false; + bool has_output = false; + DB::Chunk input_chunk; + DB::Chunk output_chunk; + + void setupOutputPort(); +}; + +}; diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp index adf25d13f29d..0dd1ac358466 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.cpp @@ -44,7 +44,7 @@ GraceAggregatingTransform::GraceAggregatingTransform( , aggregate_columns(params_->params.aggregates_size) , no_pre_aggregated(no_pre_aggregated_) , final_output(final_output_) - , tmp_data_disk(std::make_unique(context_->getTempDataOnDisk())) + , tmp_data_disk(context_->getTempDataOnDisk()) { output_header = params->getHeader(); auto config = GraceMergingAggregateConfig::loadFromContext(context); @@ -56,7 +56,9 @@ GraceAggregatingTransform::GraceAggregatingTransform( max_allowed_memory_usage_ratio = config.max_allowed_memory_usage_ratio_for_aggregate_merging; // bucket 0 is for in-memory data, it's just a placeholder. buckets.emplace(0, BufferFileStream()); - + enable_spill_test = config.enable_spill_test; + if (enable_spill_test) + buckets.emplace(1, BufferFileStream()); current_data_variants = std::make_shared(); } @@ -289,7 +291,7 @@ void GraceAggregatingTransform::addBlockIntoFileBucket(size_t bucket_index, cons file_stream.original_blocks.push_back(block); else file_stream.intermediate_blocks.push_back(block); - if (file_stream.pending_bytes > max_pending_flush_blocks_per_bucket) + if (file_stream.pending_bytes > max_pending_flush_blocks_per_bucket || (file_stream.pending_bytes && enable_spill_test)) { flushBucket(bucket_index); file_stream.pending_bytes = 0; @@ -302,10 +304,13 @@ void GraceAggregatingTransform::flushBuckets() flushBucket(i); } -static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::list & blocks) +static size_t flushBlocksInfoDisk(std::optional& file_stream, std::list & blocks) { size_t flush_bytes = 0; DB::Blocks tmp_blocks; + if (!file_stream) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "file_stream is empty"); + auto & tmp_stream = file_stream.value(); while (!blocks.empty()) { while (!blocks.empty()) @@ -322,11 +327,11 @@ static size_t flushBlocksInfoDisk(DB::TemporaryFileStream * file_stream, std::li flush_bytes += merged_block.bytes(); if (merged_block.rows()) { - file_stream->write(merged_block); + tmp_stream->write(merged_block); } } if (flush_bytes) - file_stream->flush(); + tmp_stream->flush(); return flush_bytes; } @@ -338,7 +343,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.original_blocks.empty()) { if (!file_stream.original_file_stream) - file_stream.original_file_stream = &tmp_data_disk->createStream(header); + file_stream.original_file_stream = DB::TemporaryBlockStreamHolder(header, tmp_data_disk.get()); flush_bytes += flushBlocksInfoDisk(file_stream.original_file_stream, file_stream.original_blocks); } if (!file_stream.intermediate_blocks.empty()) @@ -346,7 +351,7 @@ size_t GraceAggregatingTransform::flushBucket(size_t bucket_index) if (!file_stream.intermediate_file_stream) { auto intermediate_header = params->aggregator.getHeader(false); - file_stream.intermediate_file_stream = &tmp_data_disk->createStream(intermediate_header); + file_stream.intermediate_file_stream = DB::TemporaryBlockStreamHolder(intermediate_header, tmp_data_disk.get()); } flush_bytes += flushBlocksInfoDisk(file_stream.intermediate_file_stream, file_stream.intermediate_blocks); } @@ -373,9 +378,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.intermediate_file_stream) { buffer_file_stream.intermediate_file_stream->finishWriting(); + auto reader = buffer_file_stream.intermediate_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.intermediate_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -383,7 +389,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, false); block = {}; } - buffer_file_stream.intermediate_file_stream = nullptr; + buffer_file_stream.intermediate_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.intermediate_blocks.empty()) @@ -398,9 +404,10 @@ std::unique_ptr GraceAggregatingTransform::prepareB if (buffer_file_stream.original_file_stream) { buffer_file_stream.original_file_stream->finishWriting(); + auto reader = buffer_file_stream.original_file_stream->getReadStream(); while (true) { - auto block = buffer_file_stream.original_file_stream->read(); + auto block = reader->read(); if (!block.rows()) break; read_bytes += block.bytes(); @@ -408,7 +415,7 @@ std::unique_ptr GraceAggregatingTransform::prepareB mergeOneBlock(block, true); block = {}; } - buffer_file_stream.original_file_stream = nullptr; + buffer_file_stream.original_file_stream.reset(); total_read_disk_time += watch.elapsedMilliseconds(); } if (!buffer_file_stream.original_blocks.empty()) diff --git a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h index c2b787393a1b..c34d7714cfd4 100644 --- a/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h +++ b/cpp-ch/local-engine/Operator/GraceAggregatingTransform.h @@ -59,7 +59,7 @@ class GraceAggregatingTransform : public DB::IProcessor DB::Aggregator::AggregateColumns aggregate_columns; DB::AggregatingTransformParamsPtr params; DB::ContextPtr context; - DB::TemporaryDataOnDiskPtr tmp_data_disk; + DB::TemporaryDataOnDiskScopePtr tmp_data_disk; DB::AggregatedDataVariantsPtr current_data_variants = nullptr; size_t current_bucket_index = 0; @@ -83,9 +83,9 @@ class GraceAggregatingTransform : public DB::IProcessor /// Only be used when there is no pre-aggregated step, store the original input blocks. std::list original_blocks; /// store the intermediate result blocks. - DB::TemporaryFileStream * intermediate_file_stream = nullptr; + std::optional intermediate_file_stream; /// Only be used when there is no pre-aggregated step - DB::TemporaryFileStream * original_file_stream = nullptr; + std::optional original_file_stream; size_t pending_bytes = 0; }; std::unordered_map buckets; @@ -116,6 +116,7 @@ class GraceAggregatingTransform : public DB::IProcessor DB::BlocksList current_final_blocks; std::unique_ptr block_converter = nullptr; bool no_more_keys = false; + bool enable_spill_test = false; double per_key_memory_usage = 0; diff --git a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp index 724a02f5b296..ab87e1b39b79 100644 --- a/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp +++ b/cpp-ch/local-engine/Operator/PartitionColumnFillingTransform.cpp @@ -57,16 +57,11 @@ ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, const std::string return column_type->createColumnConst(1, value); } -//template <> -//ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, std::string partition_value); -//template <> -//ColumnPtr createFloatPartitionColumn(DataTypePtr column_type, std::string partition_value); - PartitionColumnFillingTransform::PartitionColumnFillingTransform( const DB::Block & input_, const DB::Block & output_, const String & partition_col_name_, const String & partition_col_value_) : ISimpleTransform(input_, output_, true), partition_col_name(partition_col_name_), partition_col_value(partition_col_value_) { - partition_col_type = output_.getByName(partition_col_name_).type; + partition_col_type = output_.getByName(partition_col_name_, true).type; partition_column = createPartitionColumn(); } diff --git a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp index f25e3f22ac65..d2264e24dc13 100644 --- a/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp +++ b/cpp-ch/local-engine/Operator/WindowGroupLimitStep.cpp @@ -37,7 +37,6 @@ enum class WindowGroupLimitFunction DenseRank }; - template class WindowGroupLimitTransform : public DB::IProcessor { @@ -50,7 +49,6 @@ class WindowGroupLimitTransform : public DB::IProcessor , partition_columns(partition_columns_) , sort_columns(sort_columns_) , limit(limit_) - { } ~WindowGroupLimitTransform() override = default; @@ -95,9 +93,7 @@ class WindowGroupLimitTransform : public DB::IProcessor void work() override { if (!has_input) [[unlikely]] - { return; - } DB::Block block = header.cloneWithColumns(input_chunk.getColumns()); size_t partition_start_row = 0; size_t chunk_rows = input_chunk.getNumRows(); @@ -151,7 +147,6 @@ class WindowGroupLimitTransform : public DB::IProcessor DB::Columns partition_start_row_columns; DB::Columns peer_group_start_row_columns; - size_t advanceNextPartition(const DB::Chunk & chunk, size_t start_offset) { if (partition_start_row_columns.empty()) @@ -159,12 +154,8 @@ class WindowGroupLimitTransform : public DB::IProcessor size_t max_row = chunk.getNumRows(); for (size_t i = start_offset; i < max_row; ++i) - { if (!isRowEqual(partition_columns, partition_start_row_columns, 0, chunk.getColumns(), i)) - { return i; - } - } return max_row; } @@ -199,7 +190,6 @@ class WindowGroupLimitTransform : public DB::IProcessor if (current_row_rank_value > limit) return; - size_t chunk_rows = chunk.getNumRows(); auto has_peer_group_ended = [&](size_t offset, size_t partition_end_offset, size_t chunk_rows_) { return offset < partition_end_offset || end_offset < chunk_rows_; }; @@ -241,6 +231,7 @@ class WindowGroupLimitTransform : public DB::IProcessor size_t limit_remained = limit - current_row_rank_value + 1; rows = rows > limit_remained ? limit_remained : rows; insertResultValue(chunk, start_offset, rows); + current_row_rank_value += rows; } else @@ -249,8 +240,8 @@ class WindowGroupLimitTransform : public DB::IProcessor while (peer_group_start_offset < end_offset && current_row_rank_value <= limit) { auto next_peer_group_start_offset = advanceNextPeerGroup(chunk, peer_group_start_offset, end_offset); - - insertResultValue(chunk, peer_group_start_offset, next_peer_group_start_offset - peer_group_start_offset); + size_t group_rows = next_peer_group_start_offset - peer_group_start_offset; + insertResultValue(chunk, peer_group_start_offset, group_rows); try_end_peer_group(peer_group_start_offset, next_peer_group_start_offset, end_offset, chunk_rows); peer_group_start_offset = next_peer_group_start_offset; } @@ -261,12 +252,8 @@ class WindowGroupLimitTransform : public DB::IProcessor if (!rows) return; if (output_columns.empty()) - { for (const auto & col : chunk.getColumns()) - { output_columns.push_back(col->cloneEmpty()); - } - } size_t i = 0; for (const auto & col : chunk.getColumns()) { @@ -279,12 +266,8 @@ class WindowGroupLimitTransform : public DB::IProcessor if (peer_group_start_row_columns.empty()) peer_group_start_row_columns = extractOneRowColumns(chunk, start_offset); for (size_t i = start_offset; i < partition_end_offset; ++i) - { if (!isRowEqual(sort_columns, peer_group_start_row_columns, 0, chunk.getColumns(), i)) - { return i; - } - } return partition_end_offset; } }; diff --git a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp index f2cc84082d77..49bbe02c551c 100644 --- a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp +++ b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.cpp @@ -164,6 +164,7 @@ WindowGroupOptimizationInfo WindowGroupOptimizationInfo::parse(const String & ad auto kkvs = convertToKVs(advance); auto & kvs = kkvs["WindowGroupLimitParameters"]; tryAssign(kvs, "window_function", info.window_function); + tryAssign(kvs, "is_aggregate_group_limit", info.is_aggregate_group_limit); return info; } } diff --git a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h index 3028639bb37c..795577328f99 100644 --- a/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h +++ b/cpp-ch/local-engine/Parser/AdvancedParametersParseUtil.h @@ -49,6 +49,7 @@ struct AggregateOptimizationInfo struct WindowGroupOptimizationInfo { String window_function; + bool is_aggregate_group_limit = false; static WindowGroupOptimizationInfo parse(const String & advnace); }; } diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp index 42c4230e4a9d..5452972c6f7e 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.cpp @@ -76,7 +76,7 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::parseExpression(DB::Action std::pair AggregateFunctionParser::parseLiteral(const substrait::Expression_Literal & literal) const { - return LiteralParser().parse(literal); + return LiteralParser::parse(literal); } DB::ActionsDAG::NodeRawConstPtrs @@ -195,6 +195,8 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( actions_dag.addOrReplaceInOutputs(*func_node); } + func_node = convertNanToNullIfNeed(func_info, func_node, actions_dag); + if (output_type.has_decimal()) { String checkDecimalOverflowSparkOrNull = "checkDecimalOverflowSparkOrNull"; @@ -209,6 +211,25 @@ const DB::ActionsDAG::Node * AggregateFunctionParser::convertNodeTypeIfNeeded( return func_node; } +const DB::ActionsDAG::Node * AggregateFunctionParser::convertNanToNullIfNeed( + const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag) const +{ + if (getCHFunctionName(func_info) != "corr" || !func_node->result_type->isNullable()) + return func_node; + + /// result is nullable. + /// if result is NaN, convert it to NULL. + auto is_nan_func_node = toFunctionNode(actions_dag, "isNaN", getUniqueName("isNaN"), {func_node}); + auto nullable_col = func_node->result_type->createColumn(); + nullable_col->insertDefault(); + const auto * null_node + = &actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(nullable_col), func_node->result_type, getUniqueName("null"))); + DB::ActionsDAG::NodeRawConstPtrs convert_nan_func_args = {is_nan_func_node, null_node, func_node}; + func_node = toFunctionNode(actions_dag, "if", func_node->result_name, convert_nan_func_args); + actions_dag.addOrReplaceInOutputs(*func_node); + return func_node; +} + AggregateFunctionParserFactory & AggregateFunctionParserFactory::instance() { static AggregateFunctionParserFactory factory; diff --git a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h index 02b09fc256f4..a41b3e3ad956 100644 --- a/cpp-ch/local-engine/Parser/AggregateFunctionParser.h +++ b/cpp-ch/local-engine/Parser/AggregateFunctionParser.h @@ -144,6 +144,9 @@ class AggregateFunctionParser std::pair parseLiteral(const substrait::Expression_Literal & literal) const; + const DB::ActionsDAG::Node * convertNanToNullIfNeed( + const CommonFunctionInfo & func_info, const DB::ActionsDAG::Node * func_node, DB::ActionsDAG & actions_dag) const; + ParserContextPtr parser_context; std::unique_ptr expression_parser; Poco::Logger * logger = &Poco::Logger::get("AggregateFunctionParserFactory"); diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.cpp b/cpp-ch/local-engine/Parser/ExpressionParser.cpp index ee64aff078de..e7de46483eda 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.cpp +++ b/cpp-ch/local-engine/Parser/ExpressionParser.cpp @@ -15,7 +15,7 @@ * limitations under the License. */ #include "ExpressionParser.h" -#include +#include #include #include #include @@ -31,17 +31,13 @@ #include #include #include -#include #include #include #include #include -#include #include #include -#include #include -#include "SerializedPlanParser.h" namespace DB { @@ -55,7 +51,7 @@ extern const int BAD_ARGUMENTS; namespace local_engine { -std::pair LiteralParser::parse(const substrait::Expression_Literal & literal) const +std::pair LiteralParser::parse(const substrait::Expression_Literal & literal) { DB::DataTypePtr type; DB::Field field; @@ -93,7 +89,7 @@ std::pair LiteralParser::parse(const substrait::Expr break; } case substrait::Expression_Literal::kBoolean: { - type = std::make_shared(); + type = DB::DataTypeFactory::instance().get("Bool"); field = literal.boolean() ? UInt8(1) : UInt8(0); break; } @@ -274,7 +270,7 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ case substrait::Expression::RexTypeCase::kLiteral: { DB::DataTypePtr type; DB::Field field; - std::tie(type, field) = LiteralParser().parse(rel.literal()); + std::tie(type, field) = LiteralParser::parse(rel.literal()); return addConstColumn(actions_dag, type, field); } @@ -288,77 +284,78 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ case substrait::Expression::RexTypeCase::kCast: { if (!rel.cast().has_type() || !rel.cast().has_input()) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Doesn't have type or input in cast node."); - DB::ActionsDAG::NodeRawConstPtrs args; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Doesn't have type or input in cast node."); + ActionsDAG::NodeRawConstPtrs args; - String cast_function = "CAST"; const auto & input = rel.cast().input(); args.emplace_back(parseExpression(actions_dag, input)); const auto & substrait_type = rel.cast().type(); const auto & input_type = args[0]->result_type; - DB::DataTypePtr non_nullable_input_type = DB::removeNullable(input_type); - DB::DataTypePtr output_type = TypeParser::parseType(substrait_type); - DB::DataTypePtr non_nullable_output_type = DB::removeNullable(output_type); + DataTypePtr denull_input_type = removeNullable(input_type); + DataTypePtr output_type = TypeParser::parseType(substrait_type); + DataTypePtr denull_output_type = removeNullable(output_type); - const DB::ActionsDAG::Node * function_node = nullptr; + const ActionsDAG::Node * result_node = nullptr; if (substrait_type.has_binary()) { /// Spark cast(x as BINARY) -> CH reinterpretAsStringSpark(x) - function_node = toFunctionNode(actions_dag, "reinterpretAsStringSpark", args); + result_node = toFunctionNode(actions_dag, "reinterpretAsStringSpark", args); } - else if (DB::isString(non_nullable_input_type) && DB::isDate32(non_nullable_output_type)) - function_node = toFunctionNode(actions_dag, "sparkToDate", args); - else if (DB::isString(non_nullable_input_type) && DB::isDateTime64(non_nullable_output_type)) - function_node = toFunctionNode(actions_dag, "sparkToDateTime", args); - else if (DB::isDecimal(non_nullable_input_type) && DB::isString(non_nullable_output_type)) + else if (isString(denull_input_type) && isDate32(denull_output_type)) + result_node = toFunctionNode(actions_dag, "sparkToDate", args); + else if (isString(denull_input_type) && isDateTime64(denull_output_type)) + result_node = toFunctionNode(actions_dag, "sparkToDateTime", args); + else if (isDecimal(denull_input_type) && isString(denull_output_type)) { /// Spark cast(x as STRING) if x is Decimal -> CH toDecimalString(x, scale) - UInt8 scale = DB::getDecimalScale(*non_nullable_input_type); - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), DB::Field(scale))); - function_node = toFunctionNode(actions_dag, "toDecimalString", args); + UInt8 scale = getDecimalScale(*denull_input_type); + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), Field(scale))); + result_node = toFunctionNode(actions_dag, "toDecimalString", args); } - else if (DB::isFloat(non_nullable_input_type) && DB::isInt(non_nullable_output_type)) + else if (isFloat(denull_input_type) && isInt(denull_output_type)) { - String function_name = "sparkCastFloatTo" + non_nullable_output_type->getName(); - function_node = toFunctionNode(actions_dag, function_name, args); + String function_name = "sparkCastFloatTo" + denull_output_type->getName(); + result_node = toFunctionNode(actions_dag, function_name, args); } - else if ((isDecimal(non_nullable_input_type) && substrait_type.has_decimal())) + else if ((isDecimal(denull_input_type) && substrait_type.has_decimal())) { args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().precision())); args.emplace_back(addConstColumn(actions_dag, std::make_shared(), substrait_type.decimal().scale())); - - function_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); + result_node = toFunctionNode(actions_dag, "checkDecimalOverflowSparkOrNull", args); } - else if (isMap(non_nullable_input_type) && isString(non_nullable_output_type)) + else if (isMap(denull_input_type) && isString(denull_output_type)) { // ISSUE-7389: spark cast(map to string) has different behavior with CH cast(map to string) - auto map_input_type = std::static_pointer_cast(non_nullable_input_type); + auto map_input_type = std::static_pointer_cast(denull_input_type); args.emplace_back(addConstColumn(actions_dag, map_input_type->getKeyType(), map_input_type->getKeyType()->getDefault())); - args.emplace_back(addConstColumn(actions_dag, map_input_type->getValueType(), map_input_type->getValueType()->getDefault())); - function_node = toFunctionNode(actions_dag, "sparkCastMapToString", args); + args.emplace_back( + addConstColumn(actions_dag, map_input_type->getValueType(), map_input_type->getValueType()->getDefault())); + result_node = toFunctionNode(actions_dag, "sparkCastMapToString", args); + } + else if (isString(denull_input_type) && substrait_type.has_bool_()) + { + /// cast(string to boolean) + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "accurateCastOrNull", args); + } + else if (isString(denull_input_type) && isInt(denull_output_type)) + { + /// Spark cast(x as INT) if x is String -> CH cast(trim(x) as INT) + /// Refer to https://github.com/apache/incubator-gluten/issues/4956 + args[0] = toFunctionNode(actions_dag, "trim", {args[0]}); + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "CAST", args); } else { - if (DB::isString(non_nullable_input_type) && DB::isInt(non_nullable_output_type)) - { - /// Spark cast(x as INT) if x is String -> CH cast(trim(x) as INT) - /// Refer to https://github.com/apache/incubator-gluten/issues/4956 - args[0] = toFunctionNode(actions_dag, "trim", {args[0]}); - } - else if (DB::isString(non_nullable_input_type) && substrait_type.has_bool_()) - { - /// cast(string to boolean) - cast_function = "accurateCastOrNull"; - } - - /// Common process - args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); - function_node = toFunctionNode(actions_dag, cast_function, args); + /// Common process: CAST(input, type) + args.emplace_back(addConstColumn(actions_dag, std::make_shared(), output_type->getName())); + result_node = toFunctionNode(actions_dag, "CAST", args); } - actions_dag.addOrReplaceInOutputs(*function_node); - return function_node; + actions_dag.addOrReplaceInOutputs(*result_node); + return result_node; } case substrait::Expression::RexTypeCase::kIfThen: { @@ -411,7 +408,7 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ args.emplace_back(parseExpression(actions_dag, rel.singular_or_list().value())); bool nullable = false; - int options_len = static_cast(options.size()); + int options_len = options.size(); for (int i = 0; i < options_len; ++i) { if (!options[i].has_literal()) @@ -421,15 +418,14 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ } DB::DataTypePtr elem_type; - LiteralParser literal_parser; - std::tie(elem_type, std::ignore) = literal_parser.parse(options[0].literal()); + std::tie(elem_type, std::ignore) = LiteralParser::parse(options[0].literal()); elem_type = wrapNullableType(nullable, elem_type); DB::MutableColumnPtr elem_column = elem_type->createColumn(); elem_column->reserve(options_len); for (int i = 0; i < options_len; ++i) { - auto type_and_field = LiteralParser().parse(options[i].literal()); + auto type_and_field = LiteralParser::parse(options[i].literal()); auto option_type = wrapNullableType(nullable, type_and_field.first); if (!elem_type->equals(*option_type)) throw DB::Exception( @@ -440,16 +436,12 @@ const ActionsDAG::Node * ExpressionParser::parseExpression(ActionsDAG & actions_ elem_column->insert(type_and_field.second); } - - DB::MutableColumns elem_columns; - elem_columns.emplace_back(std::move(elem_column)); - auto name = getUniqueName("__set"); - DB::Block elem_block; - elem_block.insert(DB::ColumnWithTypeAndName(nullptr, elem_type, name)); - elem_block.setColumns(std::move(elem_columns)); + ColumnWithTypeAndName elem_block{std::move(elem_column), elem_type, name}; - auto future_set = std::make_shared(elem_block, context->queryContext()->getSettingsRef()); + PreparedSets prepared_sets; + FutureSet::Hash emptyKey; + auto future_set = prepared_sets.addFromTuple(emptyKey, {elem_block}, context->queryContext()->getSettingsRef()); auto arg = DB::ColumnSet::create(1, std::move(future_set)); args.emplace_back(&actions_dag.addColumn(DB::ColumnWithTypeAndName(std::move(arg), std::make_shared(), name))); @@ -624,7 +616,7 @@ const DB::ActionsDAG::Node * ExpressionParser::toFunctionNode( return &actions_dag.addFunction(function_builder, args, result_name); } -UInt64 ExpressionParser::unique_name_counter = 0; +std::atomic ExpressionParser::unique_name_counter = 0; String ExpressionParser::getUniqueName(const String & name) const { return name + "_" + std::to_string(unique_name_counter++); diff --git a/cpp-ch/local-engine/Parser/ExpressionParser.h b/cpp-ch/local-engine/Parser/ExpressionParser.h index 5aedb94f2c35..06a80d756e3f 100644 --- a/cpp-ch/local-engine/Parser/ExpressionParser.h +++ b/cpp-ch/local-engine/Parser/ExpressionParser.h @@ -16,16 +16,16 @@ */ #pragma once +#include #include #include #include #include -#include "SerializedPlanParser.h" namespace local_engine { -struct ParserContext; +class ParserContext; class SerializedPlanParser; class LiteralParser @@ -33,17 +33,17 @@ class LiteralParser public: /// Parse a substrait literal into a CH field /// returns are the type and field value. - std::pair parse(const substrait::Expression_Literal & literal) const; + static std::pair parse(const substrait::Expression_Literal & literal); }; class ExpressionParser { public: - ExpressionParser(std::shared_ptr context_) : context(context_) { } + ExpressionParser(const std::shared_ptr & context_) : context(context_) { } ~ExpressionParser() = default; - /// Append a counter suffix to name + /// Append a counter-suffix to name String getUniqueName(const String & name) const; const DB::ActionsDAG::Node * addConstColumn(DB::ActionsDAG & actions_dag, const DB::DataTypePtr type, const DB::Field & field) const; @@ -74,7 +74,7 @@ class ExpressionParser String safeGetFunctionName(const substrait::Expression_ScalarFunction & func_) const; private: - static UInt64 unique_name_counter; + static std::atomic unique_name_counter; std::shared_ptr context; DB::ActionsDAG::NodeRawConstPtrs @@ -83,6 +83,5 @@ class ExpressionParser const substrait::Expression_ScalarFunction & func, DB::ActionsDAG & actions_dag, bool position, bool & is_map) const; DB::ActionsDAG::NodeRawConstPtrs parseJsonTuple(const substrait::Expression_ScalarFunction & func, DB::ActionsDAG & actions_dag) const; - }; } diff --git a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp index 370dc4bda9ac..2cef922ba475 100644 --- a/cpp-ch/local-engine/Parser/FunctionExecutor.cpp +++ b/cpp-ch/local-engine/Parser/FunctionExecutor.cpp @@ -79,8 +79,6 @@ void FunctionExecutor::parseExpression() /// Notice keep_result must be true, because result_node of current function must be output node in actions_dag const auto * node = expression_parser->parseFunction(expression.scalar_function(), actions_dag, true); result_name = node->result_name; - // std::cout << "actions_dag:" << std::endl; - // std::cout << actions_dag->dumpDAG() << std::endl; expression_actions = std::make_unique(std::move(actions_dag)); } @@ -115,9 +113,7 @@ bool FunctionExecutor::executeAndCompare(const std::vector FunctionParser::parseLiteral(const substrait::Expression_Literal & literal) const { - return LiteralParser().parse(literal); + return LiteralParser::parse(literal); } ActionsDAG::NodeRawConstPtrs @@ -115,33 +115,60 @@ const ActionsDAG::Node * FunctionParser::convertNodeTypeIfNeeded( const substrait::Expression_ScalarFunction & substrait_func, const ActionsDAG::Node * func_node, ActionsDAG & actions_dag) const { const auto & output_type = substrait_func.output_type(); - if (!TypeParser::isTypeMatched(output_type, func_node->result_type)) + const ActionsDAG::Node * result_node = nullptr; + + auto convert_type_if_needed = [&]() { - auto result_type = TypeParser::parseType(substrait_func.output_type()); - if (DB::isDecimalOrNullableDecimal(result_type)) + if (!TypeParser::isTypeMatched(output_type, func_node->result_type)) { - return ActionsDAGUtil::convertNodeType( - actions_dag, - func_node, - // as stated in isTypeMatched, currently we don't change nullability of the result type - func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) - : local_engine::removeNullable(result_type), - func_node->result_name, - CastType::accurateOrNull); + auto result_type = TypeParser::parseType(substrait_func.output_type()); + if (DB::isDecimalOrNullableDecimal(result_type)) + { + return ActionsDAGUtil::convertNodeType( + actions_dag, + func_node, + // as stated in isTypeMatched, currently we don't change nullability of the result type + func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, result_type) + : local_engine::removeNullable(result_type), + func_node->result_name, + CastType::accurateOrNull); + } + else + { + return ActionsDAGUtil::convertNodeType( + actions_dag, + func_node, + // as stated in isTypeMatched, currently we don't change nullability of the result type + func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, TypeParser::parseType(output_type)) + : DB::removeNullable(TypeParser::parseType(output_type)), + func_node->result_name); + } } else + return func_node; + }; + result_node = convert_type_if_needed(); + + /// Notice that in CH Bool and UInt8 have different serialization and deserialization methods, which will cause issue when executing cast(bool as string) in spark in spark. + auto convert_uint8_to_bool_if_needed = [&]() -> const auto * + { + auto * mutable_result_node = const_cast(result_node); + auto denull_result_type = DB::removeNullable(result_node->result_type); + if (isUInt8(denull_result_type) && output_type.has_bool_()) { - return ActionsDAGUtil::convertNodeType( - actions_dag, - func_node, - // as stated in isTypeMatched, currently we don't change nullability of the result type - func_node->result_type->isNullable() ? local_engine::wrapNullableType(true, TypeParser::parseType(output_type)) - : DB::removeNullable(TypeParser::parseType(output_type)), - func_node->result_name); + auto bool_type = DB::DataTypeFactory::instance().get("Bool"); + if (result_node->result_type->isNullable()) + bool_type = DB::makeNullable(bool_type); + + mutable_result_node->result_type = std::move(bool_type); + return mutable_result_node; } - } - else - return func_node; + else + return result_node; + }; + result_node = convert_uint8_to_bool_if_needed(); + + return result_node; } void FunctionParserFactory::registerFunctionParser(const String & name, Value value) diff --git a/cpp-ch/local-engine/Parser/FunctionParser.h b/cpp-ch/local-engine/Parser/FunctionParser.h index d9ca7b51285b..23216f22034b 100644 --- a/cpp-ch/local-engine/Parser/FunctionParser.h +++ b/cpp-ch/local-engine/Parser/FunctionParser.h @@ -60,6 +60,7 @@ class FunctionParser { return parseFunctionArguments(substrait_func, actions_dag); } + virtual DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const; diff --git a/cpp-ch/local-engine/Parser/LocalExecutor.cpp b/cpp-ch/local-engine/Parser/LocalExecutor.cpp index ebbf5064c92d..6ef860aff92d 100644 --- a/cpp-ch/local-engine/Parser/LocalExecutor.cpp +++ b/cpp-ch/local-engine/Parser/LocalExecutor.cpp @@ -72,7 +72,7 @@ bool LocalExecutor::hasNext() return true; } -bool LocalExecutor::fallbackMode() +bool LocalExecutor::fallbackMode() const { return executor.get() || fallback_mode; } @@ -110,7 +110,7 @@ Block * LocalExecutor::nextColumnar() return columnar_batch; } -void LocalExecutor::cancel() +void LocalExecutor::cancel() const { if (executor) executor->cancel(); @@ -118,7 +118,7 @@ void LocalExecutor::cancel() push_executor->cancel(); } -void LocalExecutor::setSinks(std::function setter) +void LocalExecutor::setSinks(const std::function & setter) const { setter(*query_pipeline_builder); } @@ -137,7 +137,7 @@ Block LocalExecutor::getHeader() LocalExecutor::LocalExecutor(QueryPlanPtr query_plan, QueryPipelineBuilderPtr pipeline_builder, bool dump_pipeline_) : query_pipeline_builder(std::move(pipeline_builder)) - , header(query_plan->getCurrentHeader().cloneEmpty()) + , header(query_pipeline_builder->getHeader().cloneEmpty()) , dump_pipeline(dump_pipeline_) , ch_column_to_spark_row(std::make_unique()) , current_query_plan(std::move(query_plan)) diff --git a/cpp-ch/local-engine/Parser/LocalExecutor.h b/cpp-ch/local-engine/Parser/LocalExecutor.h index cce6cb20a227..ea1445c605e1 100644 --- a/cpp-ch/local-engine/Parser/LocalExecutor.h +++ b/cpp-ch/local-engine/Parser/LocalExecutor.h @@ -49,11 +49,11 @@ class LocalExecutor : public BlockIterator DB::Block * nextColumnar(); bool hasNext(); - bool fallbackMode(); + bool fallbackMode() const; /// Stop execution, used when task receives shutdown command or executor receives SIGTERM signal - void cancel(); - void setSinks(std::function setter); + void cancel() const; + void setSinks(const std::function & setter) const; void execute(); DB::Block getHeader(); RelMetricPtr getMetric() const { return metric; } diff --git a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp index 269d55e64560..f5498b01ab18 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/AggregateRelParser.cpp @@ -39,19 +39,8 @@ namespace DB { namespace Setting { -extern const SettingsUInt64 max_bytes_before_external_group_by; -extern const SettingsBool optimize_group_by_constant_keys; -extern const SettingsUInt64 min_free_disk_space_for_temporary_data; -extern const SettingsMaxThreads max_threads; -extern const SettingsBool empty_result_for_aggregation_by_empty_set; -extern const SettingsUInt64 group_by_two_level_threshold_bytes; -extern const SettingsOverflowModeGroupBy group_by_overflow_mode; -extern const SettingsUInt64 max_rows_to_group_by; extern const SettingsBool enable_memory_bound_merging_of_aggregation_results; extern const SettingsUInt64 aggregation_in_order_max_block_bytes; -extern const SettingsUInt64 group_by_two_level_threshold; -extern const SettingsFloat min_hit_rate_to_use_consecutive_keys_optimization; -extern const SettingsMaxThreads max_threads; extern const SettingsUInt64 max_block_size; } namespace ErrorCodes @@ -184,9 +173,7 @@ void AggregateRelParser::setup(DB::QueryPlanPtr query_plan, const substrait::Rel agg_info.signature_function_name = *parseSignatureFunctionName(measure.measure().function_reference()); auto function_parser = AggregateFunctionParserFactory::instance().get(agg_info.signature_function_name, parser_context); if (!function_parser) - { throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported aggregate function: {}", agg_info.signature_function_name); - } /// Put function_parser, parser_func_info and function_name into agg_info for reducing repeated builds. agg_info.function_parser = function_parser; agg_info.parser_func_info = AggregateFunctionParser::CommonFunctionInfo(measure); @@ -198,16 +185,10 @@ void AggregateRelParser::setup(DB::QueryPlanPtr query_plan, const substrait::Rel if (aggregate_rel->groupings_size() == 1) { for (const auto & expr : aggregate_rel->groupings(0).grouping_expressions()) - { if (expr.has_selection() && expr.selection().has_direct_reference()) - { grouping_keys.push_back(input_header.getByPosition(expr.selection().direct_reference().struct_field().field()).name); - } else - { throw Exception(ErrorCodes::BAD_ARGUMENTS, "unsupported group expression: {}", expr.DebugString()); - } - } } else if (aggregate_rel->groupings_size() != 0) { @@ -345,23 +326,23 @@ void AggregateRelParser::addMergingAggregatedStep() AggregateDescriptions aggregate_descriptions; buildAggregateDescriptions(aggregate_descriptions); const auto & settings = getContext()->getSettingsRef(); - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_threads], - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization]); auto config = StreamingAggregateConfig::loadFromContext(getContext()); if (config.enable_streaming_aggregating) { - params.group_by_two_level_threshold = settings[Setting::group_by_two_level_threshold]; + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::PARTIAL_TO_FINISHED); auto merging_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, false); steps.emplace_back(merging_step.get()); plan->addStep(std::move(merging_step)); } else { + auto params = AggregatorParamsHelper::buildParams( + getContext(), + grouping_keys, + aggregate_descriptions, + AggregatorParamsHelper::Mode::PARTIAL_TO_FINISHED, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); /// We don't use the grouping set feature in CH, so grouping_sets_params_list should always be empty. DB::GroupingSetsParamsList grouping_sets_params_list; auto merging_step = std::make_unique( @@ -389,54 +370,20 @@ void AggregateRelParser::addCompleteModeAggregatedStep() auto config = StreamingAggregateConfig::loadFromContext(getContext()); if (config.enable_streaming_aggregating) { - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - 0, /*settings[Setting::max_bytes_before_external_group_by]*/ - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::INIT_TO_COMPLETED); auto merging_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, true); steps.emplace_back(merging_step.get()); plan->addStep(std::move(merging_step)); } else { - Aggregator::Params params( + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - settings[Setting::max_bytes_before_external_group_by], - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + AggregatorParamsHelper::Mode::INIT_TO_COMPLETED, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); auto aggregating_step = std::make_unique( plan->getCurrentHeader(), @@ -471,9 +418,7 @@ void AggregateRelParser::addAggregatingStep() { const auto & next_rel = *(rel_stack->back()); if (next_rel.rel_type_case() == substrait::Rel::RelTypeCase::kAggregate) - { is_distinct_aggreate = true; - } } if (config.enable_streaming_aggregating) @@ -484,27 +429,9 @@ void AggregateRelParser::addAggregatingStep() // unreliable. It will appear that a small hash table is converted into a two level structure, resulting in a // lot of small blocks. So we disable this condition, reamain `group_by_two_level_threshold` as the condition to // convert a single level hash table into a two level one. - Aggregator::Params params( - grouping_keys, - aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - 0, // group_by_two_level_threshold_bytes - 0, - settings[Setting::empty_result_for_aggregation_by_empty_set], - nullptr, - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, AggregatorParamsHelper::Mode::INIT_TO_PARTIAL); + if (!is_distinct_aggreate) { auto aggregating_step = std::make_unique(getContext(), plan->getCurrentHeader(), params); @@ -525,35 +452,19 @@ void AggregateRelParser::addAggregatingStep() /// We cannot use streaming aggregating strategy in step3. Otherwise it will generate multiple blocks with same n_name in them. This /// will make the result for count(distinct(n_name)) wrong. step3 must finish all inputs before it puts any block into step4. /// So we introduce GraceAggregatingStep here, it can handle mass data with high cardinality. - auto aggregating_step - = std::make_unique(getContext(), plan->getCurrentHeader(), params, has_first_stage); + auto aggregating_step = std::make_unique(getContext(), plan->getCurrentHeader(), params, has_first_stage); steps.emplace_back(aggregating_step.get()); plan->addStep(std::move(aggregating_step)); } } else { - Aggregator::Params params( + auto params = AggregatorParamsHelper::buildParams( + getContext(), grouping_keys, aggregate_descriptions, - false, - settings[Setting::max_rows_to_group_by], - settings[Setting::group_by_overflow_mode], - settings[Setting::group_by_two_level_threshold], - settings[Setting::group_by_two_level_threshold_bytes], - settings[Setting::max_bytes_before_external_group_by], - settings[Setting::empty_result_for_aggregation_by_empty_set], - getContext()->getTempDataOnDisk(), - settings[Setting::max_threads], - settings[Setting::min_free_disk_space_for_temporary_data], - true, - 3, - PODArrayUtil::adjustMemoryEfficientSize(settings[Setting::max_block_size]), - /*enable_prefetch*/ true, - /*only_merge*/ false, - settings[Setting::optimize_group_by_constant_keys], - settings[Setting::min_hit_rate_to_use_consecutive_keys_optimization], - /*StatsCollectingParams*/ {}); + AggregatorParamsHelper::Mode::INIT_TO_PARTIAL, + AggregatorParamsHelper::Algorithm::CHTwoStageAggregate); auto aggregating_step = std::make_unique( plan->getCurrentHeader(), @@ -588,12 +499,8 @@ void AggregateRelParser::addPostProjection() for (const auto & agg_info : aggregates) { for (const auto * input_node : project_actions_dag.getInputs()) - { if (input_node->result_name == agg_info.measure_column_name) - { agg_info.function_parser->convertNodeTypeIfNeeded(agg_info.parser_func_info, input_node, project_actions_dag, false); - } - } } } else if (has_complete_stage) @@ -602,12 +509,8 @@ void AggregateRelParser::addPostProjection() for (const auto & agg_info : aggregates) { for (const auto * output_node : project_actions_dag.getOutputs()) - { if (output_node->result_name == agg_info.measure_column_name) - { agg_info.function_parser->convertNodeTypeIfNeeded(agg_info.parser_func_info, output_node, project_actions_dag, true); - } - } } } if (project_actions_dag.dumpDAG() != dag_footprint) diff --git a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp index 4fef282fe49c..ae0a50d2d3f2 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/CrossRelParser.cpp @@ -17,6 +17,7 @@ #include "CrossRelParser.h" #include +#include #include #include #include @@ -37,6 +38,11 @@ namespace DB { +namespace Setting +{ +extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; +} namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -194,8 +200,16 @@ DB::QueryPlanPtr CrossRelParser::parseJoin(const substrait::CrossRel & join, DB: else { JoinPtr hash_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty()); - QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left->getCurrentHeader(), + right->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("CROSS_JOIN"); steps.emplace_back(join_step.get()); std::vector plans; @@ -242,8 +256,12 @@ void CrossRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); + table_join.setColumnsFromJoinedTable( - right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp index 8a64c445e7ab..f3d8b4ab11ec 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.cpp @@ -15,11 +15,12 @@ * limitations under the License. */ #include "ExpandRelParser.h" +#include #include #include #include +#include #include -#include #include #include #include @@ -45,11 +46,17 @@ void updateType(DB::DataTypePtr & type, const DB::DataTypePtr & new_type) } } -DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list &) +DB::QueryPlanPtr +ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) { - const auto & expand_rel = rel.expand(); - const auto & header = query_plan->getCurrentHeader(); + if (!isLazyAggregateExpand(rel.expand())) + return normalParse(std::move(query_plan), rel, rel_stack); + else + return lazyAggregateExpandParse(std::move(query_plan), rel, rel_stack); +} +ExpandField ExpandRelParser::buildExpandField(const DB::Block & header, const substrait::ExpandRel & expand_rel) +{ std::vector> expand_kinds; std::vector> expand_fields; std::vector types; @@ -82,6 +89,10 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst auto field = project_expr.selection().direct_reference().struct_field().field(); kinds.push_back(ExpandFieldKind::EXPAND_FIELD_KIND_SELECTION); fields.push_back(field); + if (field >= header.columns()) + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Field index out of range: {}, header: {}", field, header.dumpStructure()); + } updateType(types[i], header.getByPosition(field).type); const auto & name = header.getByPosition(field).name; if (names[i].empty()) @@ -123,6 +134,28 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst } ExpandField expand_field(names, types, expand_kinds, expand_fields); + return expand_field; +} + +bool ExpandRelParser::isLazyAggregateExpand(const substrait::ExpandRel & expand_rel) +{ + const auto & input_rel = expand_rel.input(); + if (input_rel.rel_type_case() != substrait::Rel::RelTypeCase::kAggregate) + return false; + const auto & aggregate_rel = input_rel.aggregate(); + for (const auto & measure : aggregate_rel.measures()) + { + if (measure.measure().phase() != substrait::AggregationPhase::AGGREGATION_PHASE_INITIAL_TO_INTERMEDIATE) + return false; + } + return true; +} + +DB::QueryPlanPtr ExpandRelParser::normalParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list &) +{ + const auto & expand_rel = rel.expand(); + const auto & header = query_plan->getCurrentHeader(); + auto expand_field = buildExpandField(header, expand_rel); auto expand_step = std::make_unique(query_plan->getCurrentHeader(), std::move(expand_field)); expand_step->setStepDescription("Expand Step"); steps.emplace_back(expand_step.get()); @@ -130,6 +163,62 @@ DB::QueryPlanPtr ExpandRelParser::parse(DB::QueryPlanPtr query_plan, const subst return query_plan; } +DB::QueryPlanPtr ExpandRelParser::lazyAggregateExpandParse( + DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack) +{ + DB::Block input_header = query_plan->getCurrentHeader(); + const auto & expand_rel = rel.expand(); + auto expand_field = buildExpandField(input_header, expand_rel); + auto aggregate_rel = rel.expand().input().aggregate(); + auto aggregate_descriptions = buildAggregations(input_header, expand_field, aggregate_rel); + + size_t grouping_keys = aggregate_rel.groupings(0).grouping_expressions_size(); + + auto expand_step + = std::make_unique(getContext(), input_header, grouping_keys, aggregate_descriptions, expand_field); + expand_step->setStepDescription("Advanced Expand Step"); + steps.emplace_back(expand_step.get()); + query_plan->addStep(std::move(expand_step)); + return query_plan; +} + +DB::AggregateDescriptions ExpandRelParser::buildAggregations( + const DB::Block & input_header, const ExpandField & expand_field, const substrait::AggregateRel & aggregate_rel) +{ + auto header = AdvancedExpandStep::buildOutputHeader(input_header, expand_field); + DB::AggregateDescriptions descriptions; + DB::ColumnsWithTypeAndName aggregate_columns; + for (const auto & col : header.getColumnsWithTypeAndName()) + { + if (typeid_cast(col.column.get())) + aggregate_columns.push_back(col); + } + + for (size_t i = 0; i < aggregate_rel.measures_size(); ++i) + { + /// The output header of the aggregate is [grouping keys] ++ [aggregation columns] + const auto & measure = aggregate_rel.measures(i); + const auto & col = aggregate_columns[i]; + DB::AggregateDescription description; + auto aggregate_col = typeid_cast(col.column.get()); + + description.column_name = col.name; + description.argument_names = {col.name}; + + auto aggregate_function = aggregate_col->getAggregateFunction(); + description.parameters = aggregate_function->getParameters(); + + // Need apply "PartialMerge" combinator for the aggregate function. + auto function_name_with_combinator = aggregate_function->getName() + "PartialMerge"; + DB::AggregateFunctionProperties aggregate_function_properties; + description.function + = getAggregateFunction(function_name_with_combinator, {col.type}, aggregate_function_properties, description.parameters); + + descriptions.emplace_back(description); + } + return descriptions; +} + void registerExpandRelParser(RelParserFactory & factory) { auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h index 7ba985a3f6d9..4a178ab08de7 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/ExpandRelParser.h @@ -16,9 +16,9 @@ */ #pragma once #include +#include #include - namespace local_engine { class SerializedPlanParser; @@ -29,7 +29,17 @@ class ExpandRelParser : public RelParser ~ExpandRelParser() override = default; DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_) override; + DB::QueryPlanPtr normalParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); + DB::QueryPlanPtr + lazyAggregateExpandParse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.expand().input(); } + +private: + bool isLazyAggregateExpand(const substrait::ExpandRel & expand_rel); + ExpandField buildExpandField(const DB::Block & header, const substrait::ExpandRel & expand_rel); + + DB::AggregateDescriptions + buildAggregations(const DB::Block & input_header, const ExpandField & expand_field, const substrait::AggregateRel & aggregate_rel); }; } diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp new file mode 100644 index 000000000000..e39878cae8f7 --- /dev/null +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.cpp @@ -0,0 +1,505 @@ +/* + * 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. + */ + +#include "GroupLimitRelParser.h" +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +} + +namespace DB +{ +namespace Setting +{ +extern const SettingsMaxThreads max_threads; + +} +} + +namespace local_engine +{ +GroupLimitRelParser::GroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +DB::QueryPlanPtr +GroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + const auto win_rel_def = rel.windowgrouplimit(); + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + if (optimization_info.is_aggregate_group_limit) + { + AggregateGroupLimitRelParser aggregate_group_limit_parser(parser_context); + auto plan = aggregate_group_limit_parser.parse(std::move(current_plan_), rel, rel_stack_); + steps = aggregate_group_limit_parser.getSteps(); + return std::move(plan); + } + else + { + WindowGroupLimitRelParser window_parser(parser_context); + auto plan = window_parser.parse(std::move(current_plan_), rel, rel_stack_); + steps = window_parser.getSteps(); + return std::move(plan); + } +} + +static std::vector parsePartitionFields(const google::protobuf::RepeatedPtrField & expressions) +{ + std::vector fields; + for (const auto & expr : expressions) + if (expr.has_selection()) + fields.push_back(static_cast(expr.selection().direct_reference().struct_field().field())); + else if (expr.has_literal()) + continue; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression: {}", expr.DebugString()); + return fields; +} + +std::vector parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields) +{ + std::vector fields; + for (const auto sort_field : sort_fields) + if (sort_field.expr().has_literal()) + continue; + else if (sort_field.expr().has_selection()) + fields.push_back(static_cast(sort_field.expr().selection().direct_reference().struct_field().field())); + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown expression: {}", sort_field.expr().DebugString()); + return fields; +} + + +WindowGroupLimitRelParser::WindowGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +DB::QueryPlanPtr +WindowGroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + const auto win_rel_def = rel.windowgrouplimit(); + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + window_function_name = optimization_info.window_function; + + current_plan = std::move(current_plan_); + + auto partition_fields = parsePartitionFields(win_rel_def.partition_expressions()); + auto sort_fields = parseSortFields(win_rel_def.sorts()); + size_t limit = static_cast(win_rel_def.limit()); + + auto window_group_limit_step = std::make_unique( + current_plan->getCurrentHeader(), window_function_name, partition_fields, sort_fields, limit); + window_group_limit_step->setStepDescription("Window group limit"); + steps.emplace_back(window_group_limit_step.get()); + current_plan->addStep(std::move(window_group_limit_step)); + + return std::move(current_plan); +} + +AggregateGroupLimitRelParser::AggregateGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) +{ +} + +// used to decide which branch +size_t selectBranchOnPartitionKeysCardinality( + const std::vector & partition_keys, double high_card_threshold, const std::list & chunks) +{ + size_t total_rows = 0; + std::unordered_set ids; + for (const auto & chunk : chunks) + { + total_rows += chunk.getNumRows(); + DB::WeakHash32 hash(chunk.getNumRows()); + const auto & cols = chunk.getColumns(); + for (auto i : partition_keys) + hash.update(cols[i]->getWeakHash32()); + const auto & data = hash.getData(); + for (size_t n = 0, sz = chunk.getNumRows(); n < sz; ++n) + ids.insert(data[n]); + } + LOG_DEBUG( + getLogger("AggregateGroupLimitRelParser"), + "Approximate distinct keys {}, total rows: {}, thrshold: {}", + ids.size(), + total_rows, + high_card_threshold); + return ids.size() * 1.0 / (total_rows + 1) <= high_card_threshold ? 0 : 1; +} + +DB::QueryPlanPtr AggregateGroupLimitRelParser::parse( + DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) +{ + // calculate window's topk by aggregation. + // 1. add a pre-projecttion. Make two tuple arguments for the aggregation function. One is the required columns for the output, the other + // is the required columns for sorting. + // 2. Collect the sorting directions for each sorting field, Let them as the aggregation function's parameters. + // 3. Add a aggregation step. + // 4. Add a post-projecttion. Explode the aggregation function's result, since the result is an array. + + current_plan = std::move(current_plan_); + input_header = current_plan->getCurrentHeader(); + win_rel_def = &rel.windowgrouplimit(); + + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def->advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + limit = static_cast(win_rel_def->limit()); + aggregate_function_name = getAggregateFunctionName(optimization_info.window_function); + + if (limit < 1) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Invalid limit: {}", limit); + + auto win_config = WindowConfig::loadFromContext(getContext()); + auto high_card_threshold = win_config.aggregate_topk_high_cardinality_threshold; + + // Aggregation doesn't perform well on high cardinality keys. We make two execution pathes here. + // - if the partition keys are low cardinality, run it by aggregation + // - if the partition keys are high cardinality, run it by window. + auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); + auto branch_in_header = current_plan->getCurrentHeader(); + auto branch_step = std::make_unique( + getContext(), + branch_in_header, + 2, + win_config.aggregate_topk_sample_rows, + [partition_fields, high_card_threshold](const std::list & chunks) -> size_t + { return selectBranchOnPartitionKeysCardinality(partition_fields, high_card_threshold, chunks); }); + branch_step->setStepDescription("Window TopK"); + steps.push_back(branch_step.get()); + current_plan->addStep(std::move(branch_step)); + + // If all partition keys are low cardinality keys, use aggregattion to get topk of each partition + auto aggregation_plan = BranchStepHelper::createSubPlan(branch_in_header, 1); + prePrejectionForAggregateArguments(*aggregation_plan); + addGroupLmitAggregationStep(*aggregation_plan); + postProjectionForExplodingArrays(*aggregation_plan); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Aggregate topk plan:\n{}", PlanUtil::explainPlan(*aggregation_plan)); + + auto window_plan = BranchStepHelper::createSubPlan(branch_in_header, 1); + addSortStep(*window_plan); + addWindowLimitStep(*window_plan); + auto convert_actions_dag = DB::ActionsDAG::makeConvertingActions( + window_plan->getCurrentHeader().getColumnsWithTypeAndName(), + aggregation_plan->getCurrentHeader().getColumnsWithTypeAndName(), + DB::ActionsDAG::MatchColumnsMode::Position); + auto convert_step = std::make_unique(window_plan->getCurrentHeader(), std::move(convert_actions_dag)); + convert_step->setStepDescription("Rename rank column name"); + window_plan->addStep(std::move(convert_step)); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Window topk plan:\n{}", PlanUtil::explainPlan(*window_plan)); + + std::vector branch_plans; + branch_plans.emplace_back(std::move(aggregation_plan)); + branch_plans.emplace_back(std::move(window_plan)); + auto unite_branches_step = std::make_unique(branch_in_header, std::move(branch_plans), 1); + unite_branches_step->setStepDescription("Unite TopK branches"); + steps.push_back(unite_branches_step.get()); + + current_plan->addStep(std::move(unite_branches_step)); + return std::move(current_plan); +} + +String AggregateGroupLimitRelParser::getAggregateFunctionName(const String & window_function_name) +{ + if (window_function_name == "row_number") + return "rowNumGroupArraySorted"; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unsupported window function: {}", window_function_name); +} + +// Build one tuple column as the aggregate function's arguments +void AggregateGroupLimitRelParser::prePrejectionForAggregateArguments(DB::QueryPlan & plan) +{ + auto projection_actions = std::make_shared(input_header.getColumnsWithTypeAndName()); + + + auto partition_fields = parsePartitionFields(win_rel_def->partition_expressions()); + std::set unique_partition_fields(partition_fields.begin(), partition_fields.end()); + DB::NameSet required_column_names; + auto build_tuple = [&](const DB::DataTypes & data_types, + const Strings & names, + const DB::ActionsDAG::NodeRawConstPtrs & elements, + const String & name_prefix, + String & result_name) + { + result_name = expression_parser->getUniqueName(name_prefix); + auto tuple = expression_parser->toFunctionNode(*projection_actions, "tuple", elements, result_name); + auto tuple_type = std::make_shared(data_types, names); + DB::ActionsDAG::NodeRawConstPtrs cast_args; + cast_args.push_back(tuple); + cast_args.push_back( + expression_parser->addConstColumn(*projection_actions, std::make_shared(), tuple_type->getName())); + tuple = expression_parser->toFunctionNode(*projection_actions, "CAST", cast_args, result_name); + projection_actions->addOrReplaceInOutputs(*tuple); + required_column_names.insert(tuple->result_name); + }; + + DB::DataTypes aggregate_data_tuple_types; + Strings aggregate_data_tuple_names; + DB::ActionsDAG::NodeRawConstPtrs aggregate_data_tuple_nodes; + for (size_t i = 0; i < input_header.columns(); ++i) + { + const auto & col = input_header.getByPosition(i); + if (unique_partition_fields.count(i)) + { + required_column_names.insert(col.name); + aggregate_grouping_keys.push_back(col.name); + } + else + { + aggregate_data_tuple_types.push_back(col.type); + aggregate_data_tuple_names.push_back(col.name); + aggregate_data_tuple_nodes.push_back(projection_actions->getInputs()[i]); + } + } + build_tuple( + aggregate_data_tuple_types, + aggregate_data_tuple_names, + aggregate_data_tuple_nodes, + "aggregate_data_tuple", + aggregate_tuple_column_name); + + projection_actions->removeUnusedActions(required_column_names); + LOG_DEBUG( + getLogger("AggregateGroupLimitRelParser"), + "Projection for building tuples for aggregate function:\n{}", + projection_actions->dumpDAG()); + + auto expression_step = std::make_unique(input_header, std::move(*projection_actions)); + expression_step->setStepDescription("Pre-projection for aggregate group limit arguments"); + plan.addStep(std::move(expression_step)); +} + +DB::AggregateDescription AggregateGroupLimitRelParser::buildAggregateDescription(DB::QueryPlan & plan) +{ + DB::AggregateDescription agg_desc; + agg_desc.column_name = aggregate_tuple_column_name; + agg_desc.argument_names = {aggregate_tuple_column_name}; + DB::Array parameters; + parameters.push_back(static_cast(limit)); + auto sort_directions = buildSQLLikeSortDescription(input_header, win_rel_def->sorts()); + parameters.push_back(sort_directions); + + auto header = plan.getCurrentHeader(); + DB::DataTypes arg_types; + arg_types.push_back(header.getByName(aggregate_tuple_column_name).type); + + DB::AggregateFunctionProperties properties; + agg_desc.function = getAggregateFunction(aggregate_function_name, arg_types, properties, parameters); + return agg_desc; +} + +void AggregateGroupLimitRelParser::addGroupLmitAggregationStep(DB::QueryPlan & plan) +{ + const auto & settings = getContext()->getSettingsRef(); + DB::AggregateDescriptions agg_descs = {buildAggregateDescription(plan)}; + auto params = AggregatorParamsHelper::buildParams( + getContext(), aggregate_grouping_keys, agg_descs, AggregatorParamsHelper::Mode::INIT_TO_COMPLETED); + auto agg_step = std::make_unique(getContext(), plan.getCurrentHeader(), params, true); + plan.addStep(std::move(agg_step)); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Plan after add group limit:\n{}", PlanUtil::explainPlan(plan)); +} + +void AggregateGroupLimitRelParser::postProjectionForExplodingArrays(DB::QueryPlan & plan) +{ + auto header = plan.getCurrentHeader(); + + /// flatten the array column. + auto agg_result_index = header.columns() - 1; + auto array_join_actions_dag = ArrayJoinHelper::applyArrayJoinOnOneColumn(header, agg_result_index); + auto new_steps = ArrayJoinHelper::addArrayJoinStep(getContext(), plan, array_join_actions_dag, false); + + auto array_join_output_header = plan.getCurrentHeader(); + DB::ActionsDAG flatten_actions_dag(array_join_output_header.getColumnsWithTypeAndName()); + DB::Names flatten_output_column_names; + for (size_t i = 0; i < array_join_output_header.columns() - 1; ++i) + { + const auto & col = array_join_output_header.getByPosition(i); + flatten_output_column_names.push_back(col.name); + } + auto last_column = array_join_output_header.getByPosition(array_join_output_header.columns() - 1); + const auto * tuple_column = typeid_cast(last_column.column.get()); + const auto * tuple_datatype = typeid_cast(last_column.type.get()); + const auto & field_names = tuple_datatype->getElementNames(); + DB::DataTypePtr tuple_index_type = std::make_shared(); + const auto * tuple_node = flatten_actions_dag.getInputs().back(); + for (size_t i = 0; i < field_names.size(); ++i) + { + DB::ActionsDAG::NodeRawConstPtrs tuple_index_args; + tuple_index_args.push_back(tuple_node); + tuple_index_args.push_back(expression_parser->addConstColumn(flatten_actions_dag, tuple_index_type, i + 1)); + const auto * field_node = expression_parser->toFunctionNode(flatten_actions_dag, "tupleElement", tuple_index_args, field_names[i]); + flatten_actions_dag.addOrReplaceInOutputs(*field_node); + flatten_output_column_names.push_back(field_node->result_name); + } + flatten_actions_dag.removeUnusedActions(flatten_output_column_names); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Actions dag for untupling aggregate result:\n{}", flatten_actions_dag.dumpDAG()); + auto flatten_expression_step = std::make_unique(plan.getCurrentHeader(), std::move(flatten_actions_dag)); + flatten_expression_step->setStepDescription("Untuple the aggregation result"); + plan.addStep(std::move(flatten_expression_step)); + + auto flatten_tuple_output_header = plan.getCurrentHeader(); + auto window_result_column = flatten_tuple_output_header.getByPosition(flatten_tuple_output_header.columns() - 1); + /// The result column is put at the end of the header. + auto output_header = input_header; + output_header.insert(window_result_column); + auto adjust_pos_actions_dag = DB::ActionsDAG::makeConvertingActions( + flatten_tuple_output_header.getColumnsWithTypeAndName(), + output_header.getColumnsWithTypeAndName(), + DB::ActionsDAG::MatchColumnsMode::Name); + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Actions dag for replacing columns:\n{}", adjust_pos_actions_dag.dumpDAG()); + auto adjust_pos_expression_step = std::make_unique(flatten_tuple_output_header, std::move(adjust_pos_actions_dag)); + adjust_pos_expression_step->setStepDescription("Adjust position of the output columns"); + plan.addStep(std::move(adjust_pos_expression_step)); +} + +void AggregateGroupLimitRelParser::addSortStep(DB::QueryPlan & plan) +{ + auto header = plan.getCurrentHeader(); + auto full_sort_descr = parseSortFields(header, win_rel_def->partition_expressions()); + auto sort_descr = parseSortFields(header, win_rel_def->sorts()); + full_sort_descr.insert(full_sort_descr.end(), sort_descr.begin(), sort_descr.end()); + + DB::SortingStep::Settings settings(*getContext()); + auto config = MemoryConfig::loadFromContext(getContext()); + double spill_mem_ratio = config.spill_mem_ratio; + settings.worth_external_sort = [spill_mem_ratio]() -> bool { return currentThreadGroupMemoryUsageRatio() > spill_mem_ratio; }; + auto sorting_step = std::make_unique(plan.getCurrentHeader(), full_sort_descr, 0, settings); + sorting_step->setStepDescription("Sorting step"); + plan.addStep(std::move(sorting_step)); +} + +static DB::WindowFrame buildWindowFrame(const std::string & ch_function_name) +{ + DB::WindowFrame frame; + // default window frame is [unbounded preceding, current row] + if (ch_function_name == "row_number") + { + frame.type = DB::WindowFrame::FrameType::ROWS; + frame.begin_type = DB::WindowFrame::BoundaryType::Offset; + frame.begin_offset = 1; + } + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow window function: {}", ch_function_name); + return frame; +} + +static DB::WindowFunctionDescription buildWindowFunctionDescription(const std::string & ch_function_name) +{ + DB::WindowFunctionDescription description; + if (ch_function_name == "row_number") + { + description.column_name = ch_function_name; + description.function_node = nullptr; + DB::AggregateFunctionProperties agg_props; + auto agg_func = RelParser::getAggregateFunction(ch_function_name, {}, agg_props, {}); + description.aggregate_function = agg_func; + } + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow window function: {}", ch_function_name); + return description; +} + + +// TODO: WindowGroupLimitStep has bad performance, need to improve it. So we still use window + filter here. +void AggregateGroupLimitRelParser::addWindowLimitStep(DB::QueryPlan & plan) +{ + google::protobuf::StringValue optimize_info_str; + optimize_info_str.ParseFromString(win_rel_def->advanced_extension().optimization().value()); + auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); + auto window_function_name = optimization_info.window_function; + + auto in_header = plan.getCurrentHeader(); + DB::WindowDescription win_descr; + win_descr.frame = buildWindowFrame(window_function_name); + win_descr.partition_by = parseSortFields(in_header, win_rel_def->partition_expressions()); + win_descr.order_by = parseSortFields(in_header, win_rel_def->sorts()); + win_descr.full_sort_description = win_descr.partition_by; + win_descr.full_sort_description.insert(win_descr.full_sort_description.end(), win_descr.order_by.begin(), win_descr.order_by.end()); + DB::WriteBufferFromOwnString ss; + ss << "partition by " << DB::dumpSortDescription(win_descr.partition_by); + ss << " order by " << DB::dumpSortDescription(win_descr.order_by); + ss << " " << win_descr.frame.toString(); + win_descr.window_name = ss.str(); + + auto win_func_description = buildWindowFunctionDescription(window_function_name); + win_descr.window_functions.push_back(win_func_description); + + auto win_step = std::make_unique(in_header, win_descr, win_descr.window_functions, false); + win_step->setStepDescription("Window (" + win_descr.window_name + ")"); + plan.addStep(std::move(win_step)); + + auto win_result_header = plan.getCurrentHeader(); + DB::ActionsDAG limit_actions_dag(win_result_header.getColumnsWithTypeAndName()); + const auto * rank_value_node = limit_actions_dag.getInputs().back(); + const auto * limit_value_node = expression_parser->addConstColumn(limit_actions_dag, std::make_shared(), limit); + const auto * cmp_node = expression_parser->toFunctionNode(limit_actions_dag, "lessOrEquals", {rank_value_node, limit_value_node}); + auto cmp_column_name = cmp_node->result_name; + limit_actions_dag.addOrReplaceInOutputs(*cmp_node); + auto filter_step = std::make_unique(win_result_header, std::move(limit_actions_dag), cmp_column_name, true); + plan.addStep(std::move(filter_step)); +} + +void registerWindowGroupLimitRelParser(RelParserFactory & factory) +{ + auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; + factory.registerBuilder(substrait::Rel::RelTypeCase::kWindowGroupLimit, builder); +} +} diff --git a/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h new file mode 100644 index 000000000000..6f570ba17934 --- /dev/null +++ b/cpp-ch/local-engine/Parser/RelParsers/GroupLimitRelParser.h @@ -0,0 +1,85 @@ +/* + * 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. + */ +#pragma once +#include +#include +#include +#include + +namespace local_engine +{ + +class GroupLimitRelParser : public RelParser +{ +public: + explicit GroupLimitRelParser(ParserContextPtr parser_context_); + ~GroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } +}; + +/// Similar to WindowRelParser. Some differences +/// 1. cannot support aggregate functions. only support window functions: row_number, rank, dense_rank +/// 2. row_number, rank and dense_rank are mapped to new variants +/// 3. the output columns don't contain window function results +class WindowGroupLimitRelParser : public RelParser +{ +public: + explicit WindowGroupLimitRelParser(ParserContextPtr parser_context_); + ~WindowGroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } + +private: + DB::QueryPlanPtr current_plan; + String window_function_name; +}; + +class AggregateGroupLimitRelParser : public RelParser +{ +public: + explicit AggregateGroupLimitRelParser(ParserContextPtr parser_context_); + ~AggregateGroupLimitRelParser() override = default; + DB::QueryPlanPtr + parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; + std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } + +private: + DB::QueryPlanPtr current_plan; + const substrait::WindowGroupLimitRel * win_rel_def; + String aggregate_function_name; + size_t limit = 0; + DB::Block input_header; + // DB::Block output_header; + DB::Names aggregate_grouping_keys; + String aggregate_tuple_column_name; + + String getAggregateFunctionName(const String & window_function_name); + + void prePrejectionForAggregateArguments(DB::QueryPlan & plan); + + void addGroupLmitAggregationStep(DB::QueryPlan & plan); + String parseSortDirections(const google::protobuf::RepeatedPtrField & sort_fields); + DB::AggregateDescription buildAggregateDescription(DB::QueryPlan & plan); + void postProjectionForExplodingArrays(DB::QueryPlan & plan); + + void addSortStep(DB::QueryPlan & plan); + void addWindowLimitStep(DB::QueryPlan & plan); +}; +} diff --git a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp index 0781614bf06c..6a5f9bc9378e 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/JoinRelParser.cpp @@ -45,6 +45,8 @@ namespace DB namespace Setting { extern const SettingsJoinAlgorithm join_algorithm; +extern const SettingsUInt64 max_block_size; +extern const SettingsUInt64 min_joined_block_size_bytes; } namespace ErrorCodes { @@ -313,8 +315,16 @@ DB::QueryPlanPtr JoinRelParser::parseJoin(const substrait::JoinRel & join, DB::Q JoinPtr smj_join = std::make_shared(table_join, right->getCurrentHeader().cloneEmpty(), -1); MultiEnum join_algorithm = context->getSettingsRef()[Setting::join_algorithm]; - QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), smj_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left->getCurrentHeader(), + right->getCurrentHeader(), + smj_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("SORT_MERGE_JOIN"); steps.emplace_back(join_step.get()); @@ -382,7 +392,11 @@ void JoinRelParser::addConvertStep(TableJoin & table_join, DB::QueryPlan & left, NameSet left_columns_set; for (const auto & col : left.getCurrentHeader().getNames()) left_columns_set.emplace(col); - table_join.setColumnsFromJoinedTable(right.getCurrentHeader().getNamesAndTypesList(), left_columns_set, getUniqueName("right") + "."); + table_join.setColumnsFromJoinedTable( + right.getCurrentHeader().getNamesAndTypesList(), + left_columns_set, + getUniqueName("right") + ".", + left.getCurrentHeader().getNamesAndTypesList()); // fix right table key duplicate NamesWithAliases right_table_alias; @@ -447,7 +461,7 @@ void JoinRelParser::collectJoinKeys( table_join.addDisjunct(); const auto & expr = join_rel.expression(); auto & join_clause = table_join.getClauses().back(); - std::list expressions_stack; + std::list expressions_stack; expressions_stack.push_back(&expr); while (!expressions_stack.empty()) { @@ -772,8 +786,16 @@ DB::QueryPlanPtr JoinRelParser::buildMultiOnClauseHashJoin( LOG_INFO(getLogger("JoinRelParser"), "multi join on clauses:\n{}", DB::TableJoin::formatClauses(table_join->getClauses())); JoinPtr hash_join = std::make_shared(table_join, right_plan->getCurrentHeader()); - QueryPlanStepPtr join_step - = std::make_unique(left_plan->getCurrentHeader(), right_plan->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left_plan->getCurrentHeader(), + right_plan->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("Multi join on clause hash join"); steps.emplace_back(join_step.get()); std::vector plans; @@ -806,8 +828,16 @@ DB::QueryPlanPtr JoinRelParser::buildSingleOnClauseHashJoin( { hash_join = std::make_shared(table_join, right_plan->getCurrentHeader().cloneEmpty()); } - QueryPlanStepPtr join_step - = std::make_unique(left_plan->getCurrentHeader(), right_plan->getCurrentHeader(), hash_join, 8192, 1, false); + QueryPlanStepPtr join_step = std::make_unique( + left_plan->getCurrentHeader(), + right_plan->getCurrentHeader(), + hash_join, + context->getSettingsRef()[Setting::max_block_size], + context->getSettingsRef()[Setting::min_joined_block_size_bytes], + 1, + /* required_output_ = */ NameSet{}, + false, + /* use_new_analyzer_ = */ false); join_step->setStepDescription("HASH_JOIN"); steps.emplace_back(join_step.get()); diff --git a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp index 20af6f83fc59..21e2dba44fa6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.cpp @@ -22,6 +22,7 @@ #include #include #include +#include namespace DB { @@ -85,40 +86,6 @@ ProjectRelParser::parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel } } -const DB::ActionsDAG::Node * ProjectRelParser::findArrayJoinNode(const ActionsDAG & actions_dag) -{ - const ActionsDAG::Node * array_join_node = nullptr; - const auto & nodes = actions_dag.getNodes(); - for (const auto & node : nodes) - { - if (node.type == ActionsDAG::ActionType::ARRAY_JOIN) - { - if (array_join_node) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expect single ARRAY JOIN node in generate rel"); - array_join_node = &node; - } - } - return array_join_node; -} - -ProjectRelParser::SplittedActionsDAGs ProjectRelParser::splitActionsDAGInGenerate(const ActionsDAG & actions_dag) -{ - SplittedActionsDAGs res; - - auto array_join_node = findArrayJoinNode(actions_dag); - std::unordered_set first_split_nodes(array_join_node->children.begin(), array_join_node->children.end()); - auto first_split_result = actions_dag.split(first_split_nodes); - res.before_array_join = std::move(first_split_result.first); - - array_join_node = findArrayJoinNode(first_split_result.second); - std::unordered_set second_split_nodes = {array_join_node}; - auto second_split_result = first_split_result.second.split(second_split_nodes); - res.array_join = std::move(second_split_result.first); - second_split_result.second.removeUnusedActions(); - res.after_array_join = std::move(second_split_result.second); - return res; -} - bool ProjectRelParser::isReplicateRows(substrait::GenerateRel rel) { auto signature = expression_parser->getFunctionNameInSignature(rel.generator().scalar_function()); @@ -164,7 +131,7 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re auto header = query_plan->getCurrentHeader(); auto actions_dag = expressionsToActionsDAG(expressions, header); - if (!findArrayJoinNode(actions_dag)) + if (!ArrayJoinHelper::findArrayJoinNode(actions_dag)) { /// If generator in generate rel is not explode/posexplode, e.g. json_tuple auto expression_step = std::make_unique(query_plan->getCurrentHeader(), std::move(actions_dag)); @@ -174,59 +141,8 @@ ProjectRelParser::parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Re } else { - /// If generator in generate rel is explode/posexplode, transform arrayJoin function to ARRAY JOIN STEP to apply max_block_size - /// which avoids OOM when several lateral view explode/posexplode is used in spark sqls - LOG_DEBUG(logger, "original actions_dag:{}", actions_dag.dumpDAG()); - auto splitted_actions_dags = splitActionsDAGInGenerate(actions_dag); - LOG_DEBUG(logger, "actions_dag before arrayJoin:{}", splitted_actions_dags.before_array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag during arrayJoin:{}", splitted_actions_dags.array_join.dumpDAG()); - LOG_DEBUG(logger, "actions_dag after arrayJoin:{}", splitted_actions_dags.after_array_join.dumpDAG()); - - auto ignore_actions_dag = [](const ActionsDAG & actions_dag_) -> bool - { - /* - We should ignore actions_dag like: - 0 : INPUT () (no column) String a - 1 : INPUT () (no column) String b - Output nodes: 0, 1 - */ - return actions_dag_.getOutputs().size() == actions_dag_.getNodes().size() - && actions_dag_.getInputs().size() == actions_dag_.getNodes().size(); - }; - - /// Pre-projection before array join - if (!ignore_actions_dag(splitted_actions_dags.before_array_join)) - { - auto step_before_array_join - = std::make_unique(query_plan->getCurrentHeader(), std::move(splitted_actions_dags.before_array_join)); - step_before_array_join->setStepDescription("Pre-projection In Generate"); - steps.emplace_back(step_before_array_join.get()); - query_plan->addStep(std::move(step_before_array_join)); - // LOG_DEBUG(logger, "plan1:{}", PlanUtil::explainPlan(*query_plan)); - } - - /// ARRAY JOIN - Names array_joined_columns{findArrayJoinNode(splitted_actions_dags.array_join)->result_name}; - ArrayJoin array_join; - array_join.columns = std::move(array_joined_columns); - array_join.is_left = generate_rel.outer(); - auto array_join_step = std::make_unique( - query_plan->getCurrentHeader(), std::move(array_join), false, getContext()->getSettingsRef()[Setting::max_block_size]); - array_join_step->setStepDescription("ARRAY JOIN In Generate"); - steps.emplace_back(array_join_step.get()); - query_plan->addStep(std::move(array_join_step)); - // LOG_DEBUG(logger, "plan2:{}", PlanUtil::explainPlan(*query_plan)); - - /// Post-projection after array join(Optional) - if (!ignore_actions_dag(splitted_actions_dags.after_array_join)) - { - auto step_after_array_join - = std::make_unique(query_plan->getCurrentHeader(), std::move(splitted_actions_dags.after_array_join)); - step_after_array_join->setStepDescription("Post-projection In Generate"); - steps.emplace_back(step_after_array_join.get()); - query_plan->addStep(std::move(step_after_array_join)); - // LOG_DEBUG(logger, "plan3:{}", PlanUtil::explainPlan(*query_plan)); - } + auto new_steps = ArrayJoinHelper::addArrayJoinStep(getContext(), *query_plan, actions_dag, generate_rel.outer()); + steps.insert(steps.end(), new_steps.begin(), new_steps.end()); } return query_plan; diff --git a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h index ce7a6faa70c0..cb6cf84f729d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/ProjectRelParser.h @@ -26,13 +26,6 @@ namespace local_engine class ProjectRelParser : public RelParser { public: - struct SplittedActionsDAGs - { - ActionsDAG before_array_join; /// Optional - ActionsDAG array_join; - ActionsDAG after_array_join; /// Optional - }; - explicit ProjectRelParser(ParserContextPtr parser_context_); ~ProjectRelParser() override = default; @@ -45,11 +38,6 @@ class ProjectRelParser : public RelParser DB::QueryPlanPtr parseProject(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); DB::QueryPlanPtr parseGenerate(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, std::list & rel_stack_); - static const DB::ActionsDAG::Node * findArrayJoinNode(const ActionsDAG & actions_dag); - - /// Split actions_dag of generate rel into 3 parts: before array join + during array join + after array join - static SplittedActionsDAGs splitActionsDAGInGenerate(const ActionsDAG & actions_dag); - bool isReplicateRows(substrait::GenerateRel rel); DB::QueryPlanPtr parseReplicateRows(QueryPlanPtr query_plan, substrait::GenerateRel generate_rel); diff --git a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp index 75e6e14c4a27..2a98db344f3c 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/ReadRelParser.cpp @@ -30,7 +30,7 @@ #include #include #include - +#include namespace DB { @@ -77,7 +77,7 @@ DB::QueryPlanPtr ReadRelParser::parse(DB::QueryPlanPtr query_plan, const substra else { extension_table = BinaryToMessage(split_info); - logDebugMessage(extension_table, "extension_table"); + debug::dumpMessage(extension_table, "extension_table"); } MergeTreeRelParser mergeTreeParser(parser_context, getContext()); query_plan = mergeTreeParser.parseReadRel(std::make_unique(), read, extension_table); @@ -131,7 +131,7 @@ QueryPlanStepPtr ReadRelParser::parseReadRelWithLocalFile(const substrait::ReadR else { local_files = BinaryToMessage(split_info); - logDebugMessage(local_files, "local_files"); + debug::dumpMessage(local_files, "local_files"); } auto source = std::make_shared(getContext(), header, local_files); auto source_pipe = Pipe(source); diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp new file mode 100644 index 000000000000..70cacf0633a2 --- /dev/null +++ b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.cpp @@ -0,0 +1,105 @@ +/* + * 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. + */ +#include "SortParsingUtils.h" +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ +extern const int BAD_ARGUMENTS; +extern const int LOGICAL_ERROR; +} + +namespace local_engine +{ +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & expressions) +{ + DB::SortDescription description; + for (const auto & expr : expressions) + if (expr.has_selection()) + { + auto pos = expr.selection().direct_reference().struct_field().field(); + const auto & col_name = header.getByPosition(pos).name; + description.push_back(DB::SortColumnDescription(col_name, 1, -1)); + } + else if (expr.has_literal()) + continue; + else + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression as sort field: {}", expr.DebugString()); + return description; +} + +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields) +{ + static std::map> direction_map = {{1, {1, -1}}, {2, {1, 1}}, {3, {-1, 1}}, {4, {-1, -1}}}; + + DB::SortDescription sort_descr; + for (int i = 0, sz = sort_fields.size(); i < sz; ++i) + { + const auto & sort_field = sort_fields[i]; + /// There is no meaning to sort a const column. + if (sort_field.expr().has_literal()) + continue; + + if (!sort_field.expr().has_selection() || !sort_field.expr().selection().has_direct_reference() + || !sort_field.expr().selection().direct_reference().has_struct_field()) + { + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsupport sort field"); + } + auto field_pos = sort_field.expr().selection().direct_reference().struct_field().field(); + + auto direction_iter = direction_map.find(sort_field.direction()); + if (direction_iter == direction_map.end()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsuppor sort direction: {}", sort_field.direction()); + const auto & col_name = header.getByPosition(field_pos).name; + sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); + } + return sort_descr; +} + +std::string +buildSQLLikeSortDescription(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields) +{ + static const std::unordered_map order_directions + = {{1, " asc nulls first"}, {2, " asc nulls last"}, {3, " desc nulls first"}, {4, " desc nulls last"}}; + size_t n = 0; + DB::WriteBufferFromOwnString ostr; + for (const auto & sort_field : sort_fields) + { + auto it = order_directions.find(sort_field.direction()); + if (it == order_directions.end()) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow sort direction: {}", sort_field.direction()); + if (!sort_field.expr().has_selection()) + { + throw DB::Exception( + DB::ErrorCodes::BAD_ARGUMENTS, "Sort field must be a column reference. but got {}", sort_field.DebugString()); + } + auto ref = sort_field.expr().selection().direct_reference().struct_field().field(); + const auto & col_name = header.getByPosition(ref).name; + if (n) + ostr << String(","); + // the col_name may contain '#' which can may ch fail to parse. + ostr << "`" << col_name << "`" << it->second; + n += 1; + } + LOG_DEBUG(getLogger("AggregateGroupLimitRelParser"), "Order by clasue: {}", ostr.str()); + return ostr.str(); +} +} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.h similarity index 57% rename from cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp rename to cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.h index 920fe1b9c9cc..c460fa758b6d 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionGreatest.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/SortParsingUtils.h @@ -14,25 +14,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include + +#pragma once +#include +#include +#include +#include namespace local_engine { -class SparkFunctionGreatest : public FunctionGreatestestLeast -{ -public: - static constexpr auto name = "sparkGreatest"; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionGreatest() = default; - ~SparkFunctionGreatest() override = default; - String getName() const override - { - return name; - } -}; +// convert expressions into sort description +DB::SortDescription +parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & expressions); +DB::SortDescription parseSortFields(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields); -REGISTER_FUNCTION(SparkGreatest) -{ - factory.registerFunction(); -} +std::string +buildSQLLikeSortDescription(const DB::Block & header, const google::protobuf::RepeatedPtrField & sort_fields); } diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp index 1ed4f2565dea..a8023a98185d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.cpp @@ -16,6 +16,7 @@ */ #include "SortRelParser.h" +#include #include #include #include @@ -41,7 +42,7 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st { size_t limit = parseLimit(rel_stack_); const auto & sort_rel = rel.sort(); - auto sort_descr = parseSortDescription(sort_rel.sorts(), query_plan->getCurrentHeader()); + auto sort_descr = parseSortFields(query_plan->getCurrentHeader(), sort_rel.sorts()); SortingStep::Settings settings(*getContext()); auto config = MemoryConfig::loadFromContext(getContext()); double spill_mem_ratio = config.spill_mem_ratio; @@ -53,46 +54,6 @@ SortRelParser::parse(DB::QueryPlanPtr query_plan, const substrait::Rel & rel, st return query_plan; } -DB::SortDescription -SortRelParser::parseSortDescription(const google::protobuf::RepeatedPtrField & sort_fields, const DB::Block & header) -{ - static std::map> direction_map = {{1, {1, -1}}, {2, {1, 1}}, {3, {-1, 1}}, {4, {-1, -1}}}; - - DB::SortDescription sort_descr; - for (int i = 0, sz = sort_fields.size(); i < sz; ++i) - { - const auto & sort_field = sort_fields[i]; - /// There is no meaning to sort a const column. - if (sort_field.expr().has_literal()) - continue; - - if (!sort_field.expr().has_selection() || !sort_field.expr().selection().has_direct_reference() - || !sort_field.expr().selection().direct_reference().has_struct_field()) - { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsupport sort field"); - } - auto field_pos = sort_field.expr().selection().direct_reference().struct_field().field(); - - auto direction_iter = direction_map.find(sort_field.direction()); - if (direction_iter == direction_map.end()) - { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unsuppor sort direction: {}", sort_field.direction()); - } - if (header.columns()) - { - const auto & col_name = header.getByPosition(field_pos).name; - sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); - sort_descr.back().column_name = col_name; - } - else - { - const auto & col_name = header.getByPosition(field_pos).name; - sort_descr.emplace_back(col_name, direction_iter->second.first, direction_iter->second.second); - } - } - return sort_descr; -} - size_t SortRelParser::parseLimit(std::list & rel_stack_) { if (rel_stack_.empty()) diff --git a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h index 7e6119095c75..27cd4497d19d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/SortRelParser.h @@ -30,8 +30,6 @@ class SortRelParser : public RelParser DB::QueryPlanPtr parse(DB::QueryPlanPtr query_plan, const substrait::Rel & sort_rel, std::list & rel_stack_) override; - static DB::SortDescription - parseSortDescription(const google::protobuf::RepeatedPtrField & sort_fields, const DB::Block & header); std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.sort().input(); } diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp deleted file mode 100644 index e82d68c1d115..000000000000 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.cpp +++ /dev/null @@ -1,92 +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. - */ - -#include "WindowGroupLimitRelParser.h" -#include -#include -#include -#include -#include - -namespace DB::ErrorCodes -{ -extern const int BAD_ARGUMENTS; -} - -namespace local_engine -{ -WindowGroupLimitRelParser::WindowGroupLimitRelParser(ParserContextPtr parser_context_) : RelParser(parser_context_) -{ -} - -DB::QueryPlanPtr -WindowGroupLimitRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) -{ - const auto win_rel_def = rel.windowgrouplimit(); - google::protobuf::StringValue optimize_info_str; - optimize_info_str.ParseFromString(win_rel_def.advanced_extension().optimization().value()); - auto optimization_info = WindowGroupOptimizationInfo::parse(optimize_info_str.value()); - window_function_name = optimization_info.window_function; - - current_plan = std::move(current_plan_); - - auto partition_fields = parsePartitoinFields(win_rel_def.partition_expressions()); - auto sort_fields = parseSortFields(win_rel_def.sorts()); - size_t limit = static_cast(win_rel_def.limit()); - - auto window_group_limit_step = std::make_unique( - current_plan->getCurrentHeader(), window_function_name, partition_fields, sort_fields, limit); - window_group_limit_step->setStepDescription("Window group limit"); - steps.emplace_back(window_group_limit_step.get()); - current_plan->addStep(std::move(window_group_limit_step)); - - return std::move(current_plan); -} - -std::vector -WindowGroupLimitRelParser::parsePartitoinFields(const google::protobuf::RepeatedPtrField & expressions) -{ - std::vector fields; - for (const auto & expr : expressions) - if (expr.has_selection()) - fields.push_back(static_cast(expr.selection().direct_reference().struct_field().field())); - else if (expr.has_literal()) - continue; - else - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknow expression: {}", expr.DebugString()); - return fields; -} - -std::vector WindowGroupLimitRelParser::parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields) -{ - std::vector fields; - for (const auto sort_field : sort_fields) - if (sort_field.expr().has_literal()) - continue; - else if (sort_field.expr().has_selection()) - fields.push_back(static_cast(sort_field.expr().selection().direct_reference().struct_field().field())); - else - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Unknown expression: {}", sort_field.expr().DebugString()); - return fields; -} - -void registerWindowGroupLimitRelParser(RelParserFactory & factory) -{ - auto builder = [](ParserContextPtr parser_context) { return std::make_shared(parser_context); }; - factory.registerBuilder(substrait::Rel::RelTypeCase::kWindowGroupLimit, builder); -} -} diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h deleted file mode 100644 index 573b7fad7cf9..000000000000 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowGroupLimitRelParser.h +++ /dev/null @@ -1,46 +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. - */ -#pragma once -#include -#include -#include -#include -#include - -namespace local_engine -{ -/// Similar to WindowRelParser. Some differences -/// 1. cannot support aggregate functions. only support window functions: row_number, rank, dense_rank -/// 2. row_number, rank and dense_rank are mapped to new variants -/// 3. the output columns don't contain window function results -class WindowGroupLimitRelParser : public RelParser -{ -public: - explicit WindowGroupLimitRelParser(ParserContextPtr parser_context_); - ~WindowGroupLimitRelParser() override = default; - DB::QueryPlanPtr - parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & rel, std::list & rel_stack_) override; - std::optional getSingleInput(const substrait::Rel & rel) override { return &rel.windowgrouplimit().input(); } - -private: - DB::QueryPlanPtr current_plan; - String window_function_name; - - std::vector parsePartitoinFields(const google::protobuf::RepeatedPtrField & expressions); - std::vector parseSortFields(const google::protobuf::RepeatedPtrField & sort_fields); -}; -} diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp index d52f2543c852..d3ed22cbb827 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include #include @@ -97,8 +98,8 @@ DB::WindowDescription WindowRelParser::parseWindowDescription(const WindowInfo & { DB::WindowDescription win_descr; win_descr.frame = parseWindowFrame(win_info); - win_descr.partition_by = parsePartitionBy(win_info.partition_exprs); - win_descr.order_by = SortRelParser::parseSortDescription(win_info.sort_fields, current_plan->getCurrentHeader()); + win_descr.partition_by = parseSortFields(current_plan->getCurrentHeader(), win_info.partition_exprs); + win_descr.order_by = parseSortFields(current_plan->getCurrentHeader(), win_info.sort_fields); win_descr.full_sort_description = win_descr.partition_by; win_descr.full_sort_description.insert(win_descr.full_sort_description.end(), win_descr.order_by.begin(), win_descr.order_by.end()); @@ -177,17 +178,11 @@ WindowRelParser::parseWindowFrameType(const std::string & function_name, const s frame_type = window_function.window_type(); if (frame_type == substrait::ROWS) - { return DB::WindowFrame::FrameType::ROWS; - } else if (frame_type == substrait::RANGE) - { return DB::WindowFrame::FrameType::RANGE; - } else - { throw DB::Exception(DB::ErrorCodes::UNKNOWN_TYPE, "Unknow window frame type:{}", frame_type); - } } void WindowRelParser::parseBoundType( @@ -206,13 +201,9 @@ void WindowRelParser::parseBoundType( bound_type = DB::WindowFrame::BoundaryType::Offset; preceding_direction = preceding.offset() >= 0; if (preceding.offset() < 0) - { offset = 0 - preceding.offset(); - } else - { offset = preceding.offset(); - } } else if (bound.has_following()) { @@ -220,13 +211,9 @@ void WindowRelParser::parseBoundType( bound_type = DB::WindowFrame::BoundaryType::Offset; preceding_direction = following.offset() < 0; if (following.offset() < 0) - { offset = 0 - following.offset(); - } else - { offset = following.offset(); - } } else if (bound.has_current_row()) { @@ -252,31 +239,6 @@ void WindowRelParser::parseBoundType( } } -DB::SortDescription WindowRelParser::parsePartitionBy(const google::protobuf::RepeatedPtrField & expressions) -{ - DB::Block header = current_plan->getCurrentHeader(); - DB::SortDescription sort_descr; - for (const auto & expr : expressions) - { - if (expr.has_selection()) - { - auto pos = expr.selection().direct_reference().struct_field().field(); - auto col_name = header.getByPosition(pos).name; - sort_descr.push_back(DB::SortColumnDescription(col_name, 1, 1)); - } - else if (expr.has_literal()) - { - // literal is a special case, see in #2586 - continue; - } - else - { - throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unknow partition argument type: {}", expr.DebugString()); - } - } - return sort_descr; -} - WindowFunctionDescription WindowRelParser::parseWindowFunctionDescription( const String & ch_function_name, const substrait::Expression::WindowFunction & window_function, @@ -370,8 +332,7 @@ void WindowRelParser::tryAddProjectionAfterWindow() { ActionsDAG convert_action = ActionsDAG::makeConvertingActions( current_header.getColumnsWithTypeAndName(), output_header.getColumnsWithTypeAndName(), DB::ActionsDAG::MatchColumnsMode::Name); - QueryPlanStepPtr convert_step - = std::make_unique(current_plan->getCurrentHeader(), std::move(convert_action)); + QueryPlanStepPtr convert_step = std::make_unique(current_plan->getCurrentHeader(), std::move(convert_action)); convert_step->setStepDescription("Convert window Output"); steps.emplace_back(convert_step.get()); current_plan->addStep(std::move(convert_step)); diff --git a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h index da058af857a9..fccdb1c6f4a6 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WindowRelParser.h @@ -78,7 +78,6 @@ class WindowRelParser : public RelParser DB::WindowFrame::BoundaryType & bound_type, Field & offset, bool & preceding); - DB::SortDescription parsePartitionBy(const google::protobuf::RepeatedPtrField & expressions); DB::WindowFunctionDescription parseWindowFunctionDescription( const String & ch_function_name, const substrait::Expression::WindowFunction & window_function, diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp index 569e736b3890..a76b4d398d97 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.cpp @@ -21,53 +21,57 @@ #include #include #include +#include #include +#include #include +#include +#include +#include #include #include #include #include -#include +#include #include +namespace DB::Setting +{ +extern const SettingsUInt64 min_insert_block_size_rows; +extern const SettingsUInt64 min_insert_block_size_bytes; +} + using namespace local_engine; using namespace DB; +namespace +{ DB::ProcessorPtr make_sink( const DB::ContextPtr & context, const DB::Names & partition_by, const DB::Block & input_header, const DB::Block & output_header, const std::string & base_path, - const std::string & filename, + const FileNameGenerator & generator, const std::string & format_hint, - const std::shared_ptr & stats) + const std::shared_ptr & stats) { - if (partition_by.empty()) + bool no_bucketed = !SparkPartitionedBaseSink::isBucketedWrite(input_header); + if (partition_by.empty() && no_bucketed) { - auto file_sink = std::make_shared(context, base_path, "", filename, format_hint, input_header); - file_sink->setStats(stats); - return file_sink; + return std::make_shared( + context, base_path, "", false, generator.generate(), format_hint, input_header, stats, DeltaStats{input_header.columns()}); } - auto file_sink = std::make_shared( - context, partition_by, input_header, output_header, base_path, filename, format_hint); - file_sink->setStats(stats); - return file_sink; -} - -bool need_fix_tuple(const DB::DataTypePtr & input, const DB::DataTypePtr & output) -{ - const auto original = typeid_cast(input.get()); - const auto output_type = typeid_cast(output.get()); - return original != nullptr && output_type != nullptr && !original->equals(*output_type); + return std::make_shared( + context, partition_by, input_header, output_header, base_path, generator, format_hint, stats); } DB::ExpressionActionsPtr create_rename_action(const DB::Block & input, const DB::Block & output) { DB::NamesWithAliases aliases; - for (auto ouput_name = output.begin(), input_iter = input.begin(); ouput_name != output.end(); ++ouput_name, ++input_iter) - aliases.emplace_back(DB::NameWithAlias(input_iter->name, ouput_name->name)); + for (auto output_name = output.begin(), input_iter = input.begin(); output_name != output.end(); ++output_name, ++input_iter) + aliases.emplace_back(DB::NameWithAlias(input_iter->name, output_name->name)); ActionsDAG actions_dag{blockToNameAndTypeList(input)}; actions_dag.project(aliases); @@ -138,54 +142,62 @@ void adjust_output(const DB::QueryPipelineBuilderPtr & builder, const DB::Block }); } -namespace local_engine +void addMergeTreeSinkTransform( + const DB::ContextPtr & context, + const DB::QueryPipelineBuilderPtr & builder, + const MergeTreeTable & merge_tree_table, + const DB::Block & header, + const DB::Names & partition_by) { + Chain chain; + // + auto stats = std::make_shared(header); + chain.addSink(stats); + // -IMPLEMENT_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) + SparkMergeTreeWriteSettings write_settings{context}; + if (partition_by.empty()) + write_settings.partition_settings.partition_dir = SubstraitFileSink::NO_PARTITION_ID; -void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) + auto sink = partition_by.empty() + ? SparkMergeTreeSink::create(merge_tree_table, write_settings, context->getGlobalContext(), {stats}) + : std::make_shared(header, partition_by, merge_tree_table, write_settings, context, stats); + + chain.addSource(sink); + const DB::Settings & settings = context->getSettingsRef(); + chain.addSource(std::make_shared( + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + chain.addSource(std::make_shared( + header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); + + builder->addChain(std::move(chain)); +} + +void addNormalFileWriterSinkTransform( + const DB::ContextPtr & context, + const DB::QueryPipelineBuilderPtr & builder, + const std::string & format_hint, + const DB::Block & output, + const DB::Names & partitionCols) { GlutenWriteSettings write_settings = GlutenWriteSettings::get(context); if (write_settings.task_write_tmp_dir.empty()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject temp directory."); - if (write_settings.task_write_filename.empty()) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file name."); - - assert(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - - //TODO : set compression codec according to format - assert(config["isSnappy"] == "1"); - assert(config.contains("format")); - - assert(write_rel.has_table_schema()); - const substrait::NamedStruct & table_schema = write_rel.table_schema(); - auto blockHeader = TypeParser::buildBlockFromNamedStruct(table_schema); - const auto partitionCols = collect_partition_cols(blockHeader, table_schema); + if (write_settings.task_write_filename_pattern.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Write Pipeline need inject file pattern."); - auto stats = std::make_shared(blockHeader); + FileNameGenerator generator(write_settings.task_write_filename_pattern); - adjust_output(builder, blockHeader); + auto stats = WriteStats::create(output, partitionCols); builder->addSimpleTransform( [&](const Block & cur_header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipelineBuilder::StreamType::Main) return nullptr; - return make_sink( - context, - partitionCols, - cur_header, - blockHeader, - write_settings.task_write_tmp_dir, - write_settings.task_write_filename, - config["format"], - stats); + return make_sink(context, partitionCols, cur_header, output, write_settings.task_write_tmp_dir, generator, format_hint, stats); }); builder->addSimpleTransform( [&](const Block &, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -195,30 +207,40 @@ void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel return stats; }); } +} -std::map parse_write_parameter(const std::string & input) +namespace local_engine { - std::map reuslt; - const std::string prefix = "WriteParameters:"; - const size_t prefix_pos = input.find(prefix); - if (prefix_pos == std::string::npos) - return reuslt; - const size_t start_pos = prefix_pos + prefix.length(); - const size_t end_pos = input.find('\n', start_pos); +IMPLEMENT_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) - if (end_pos == std::string::npos) - return reuslt; +void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder) +{ + assert(write_rel.has_named_table()); + const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - for (const Poco::StringTokenizer tok(input.substr(start_pos, end_pos - start_pos), ";", Poco::StringTokenizer::TOK_TRIM); - const auto & parameter : tok) + local_engine::Write write; + if (!named_table.advanced_extension().optimization().UnpackTo(&write)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to unpack write optimization with local_engine::Write."); + assert(write.has_common()); + const substrait::NamedStruct & table_schema = write_rel.table_schema(); + auto output = TypeParser::buildBlockFromNamedStruct(table_schema); + adjust_output(builder, output); + const auto partitionCols = collect_partition_cols(output, table_schema); + if (write.has_mergetree()) { - const size_t pos = parameter.find('='); - if (pos == std::string::npos) - continue; - reuslt[parameter.substr(0, pos)] = parameter.substr(pos + 1); + local_engine::MergeTreeTable merge_tree_table(write, table_schema); + GlutenWriteSettings write_settings = GlutenWriteSettings::get(context); + if (write_settings.task_write_tmp_dir.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "MergeTree Write Pipeline need inject relative path."); + if (!merge_tree_table.relative_path.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Non empty relative path for MergeTree table in pipeline mode."); + + merge_tree_table.relative_path = write_settings.task_write_tmp_dir; + addMergeTreeSinkTransform(context, builder, merge_tree_table, output, partitionCols); } - return reuslt; + else + addNormalFileWriterSinkTransform(context, builder, write.common().format(), output, partitionCols); } DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_) diff --git a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h index b083d2f637d2..01e0dabaaa7d 100644 --- a/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h +++ b/cpp-ch/local-engine/Parser/RelParsers/WriteRelParser.h @@ -40,13 +40,11 @@ namespace local_engine void addSinkTransform(const DB::ContextPtr & context, const substrait::WriteRel & write_rel, const DB::QueryPipelineBuilderPtr & builder); -/// Visible for UTs -std::map parse_write_parameter(const std::string & input); DB::Names collect_partition_cols(const DB::Block & header, const substrait::NamedStruct & struct_); -#define WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ - M(String, task_write_tmp_dir, , "The temporary directory for writing data", UNIQ) \ - M(String, task_write_filename, , "The filename for writing data", UNIQ) +#define WRITE_RELATED_SETTINGS(M, ALIAS) \ + M(String, task_write_tmp_dir, , "The temporary directory for writing data") \ + M(String, task_write_filename_pattern, , "The pattern to generate file name for writing delta parquet in spark 3.5") DECLARE_GLUTEN_SETTINGS(GlutenWriteSettings, WRITE_RELATED_SETTINGS) diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index 9799933b3385..820a99ad3bfb 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -20,35 +20,21 @@ #include #include #include -#include +#include #include #include -#include #include #include #include #include -#include -#include -#include -#include -#include #include -#include -#include -#include #include -#include #include -#include -#include #include #include #include -#include #include #include -#include #include #include #include @@ -73,6 +59,7 @@ #include #include #include +#include #include #include #include @@ -115,19 +102,24 @@ std::string join(const ActionsDAG::NodeRawConstPtrs & v, char c) return res; } -void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel & root_rel) +void SerializedPlanParser::adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan) { + const substrait::PlanRel & root_rel = plan.relations().at(0); if (root_rel.root().names_size()) { ActionsDAG actions_dag{blockToNameAndTypeList(query_plan->getCurrentHeader())}; NamesWithAliases aliases; - auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); + const auto cols = query_plan->getCurrentHeader().getNamesAndTypesList(); if (cols.getNames().size() != static_cast(root_rel.root().names_size())) + { + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, - "Missmatch result columns size. plan column size {}, subtrait plan size {}.", + "Missmatch result columns size. plan column size {}, subtrait plan name size {}.", cols.getNames().size(), root_rel.root().names_size()); + } for (int i = 0; i < static_cast(cols.getNames().size()); i++) aliases.emplace_back(NameWithAlias(cols.getNames()[i], root_rel.root().names(i))); actions_dag.project(aliases); @@ -140,48 +132,60 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel const auto & output_schema = root_rel.root().output_schema(); if (output_schema.types_size()) { - auto original_header = query_plan->getCurrentHeader(); - const auto & original_cols = original_header.getColumnsWithTypeAndName(); - if (static_cast(output_schema.types_size()) != original_cols.size()) + auto origin_header = query_plan->getCurrentHeader(); + const auto & origin_columns = origin_header.getColumnsWithTypeAndName(); + + if (static_cast(output_schema.types_size()) != origin_columns.size()) { + debug::dumpPlan(*query_plan, "clickhouse plan", true); + debug::dumpMessage(plan, "substrait::Plan", true); throw Exception( ErrorCodes::LOGICAL_ERROR, - "Mismatch output schema. plan column size {} [header: '{}'], subtrait plan size {}[schema: {}].", - original_cols.size(), - original_header.dumpStructure(), + "Missmatch result columns size. plan column size {}, subtrait plan output schema size {}, subtrait plan name size {}.", + origin_columns.size(), output_schema.types_size(), - dumpMessage(output_schema)); + root_rel.root().names_size()); } + bool need_final_project = false; - ColumnsWithTypeAndName final_cols; + ColumnsWithTypeAndName final_columns; for (int i = 0; i < output_schema.types_size(); ++i) { - const auto & col = original_cols[i]; - auto type = TypeParser::parseType(output_schema.types(i)); - // At present, we only check nullable mismatch. - // intermediate aggregate data is special, no check here. - if (type->isNullable() != col.type->isNullable() && !typeid_cast(col.type.get())) + const auto & origin_column = origin_columns[i]; + const auto & origin_type = origin_column.type; + auto final_type = TypeParser::parseType(output_schema.types(i)); + + /// Intermediate aggregate data is special, no check here. + if (typeid_cast(origin_column.type.get()) || origin_type->equals(*final_type)) + final_columns.push_back(origin_column); + else { - if (type->isNullable()) + need_final_project = true; + if (origin_column.column && isColumnConst(*origin_column.column)) { - auto wrapped = wrapNullableType(true, col.type); - final_cols.emplace_back(type->createColumn(), wrapped, col.name); - need_final_project = !wrapped->equals(*col.type); + /// For const column, we need to cast it individually. Otherwise, the const column will be converted to full column in + /// ActionsDAG::makeConvertingActions. + /// Note: creating fianl_column with Field of origin_column will cause Exception in some case. + const DB::ContextPtr context = DB::CurrentThread::get().getQueryContext(); + const FunctionOverloadResolverPtr & cast_resolver = FunctionFactory::instance().get("CAST", context); + const DataTypePtr string_type = std::make_shared(); + ColumnWithTypeAndName to_type_column = {string_type->createColumnConst(1, final_type->getName()), string_type, "__cast_const__"}; + FunctionBasePtr cast_function = cast_resolver->build({origin_column, to_type_column}); + ColumnPtr const_col = ColumnConst::create(cast_function->execute({origin_column, to_type_column}, final_type, 1, false), 1); + ColumnWithTypeAndName final_column(const_col, final_type, origin_column.name); + final_columns.emplace_back(std::move(final_column)); } else { - final_cols.emplace_back(type->createColumn(), removeNullable(col.type), col.name); - need_final_project = true; + ColumnWithTypeAndName final_column(final_type->createColumn(), final_type, origin_column.name); + final_columns.emplace_back(std::move(final_column)); } } - else - { - final_cols.push_back(col); - } } + if (need_final_project) { - ActionsDAG final_project = ActionsDAG::makeConvertingActions(original_cols, final_cols, ActionsDAG::MatchColumnsMode::Position); + ActionsDAG final_project = ActionsDAG::makeConvertingActions(origin_columns, final_columns, ActionsDAG::MatchColumnsMode::Position, true); QueryPlanStepPtr final_project_step = std::make_unique(query_plan->getCurrentHeader(), std::move(final_project)); final_project_step->setStepDescription("Project for output schema"); @@ -192,7 +196,7 @@ void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::PlanRel QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) { - logDebugMessage(plan, "substrait plan"); + debug::dumpMessage(plan, "substrait::Plan"); //parseExtensions(plan.extensions()); if (plan.relations_size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "too many relations found"); @@ -207,18 +211,13 @@ QueryPlanPtr SerializedPlanParser::parse(const substrait::Plan & plan) std::list rel_stack; auto query_plan = parseOp(first_read_rel, rel_stack); if (!writePipeline) - adjustOutput(query_plan, root_rel); + adjustOutput(query_plan, plan); #ifndef NDEBUG PlanUtil::checkOuputType(*query_plan); #endif - if (auto * logger = &Poco::Logger::get("SerializedPlanParser"); logger->debug()) - { - auto out = PlanUtil::explainPlan(*query_plan); - LOG_DEBUG(logger, "clickhouse plan:\n{}", out); - } - + debug::dumpPlan(*query_plan); return query_plan; } @@ -232,7 +231,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list auto rel_parser = RelParserFactory::instance().getBuilder(rel.rel_type_case())(parser_context); auto all_input_rels = rel_parser->getInputs(rel); - assert(all_input_rels.size() == 0 || all_input_rels.size() == 1 || all_input_rels.size() == 2); + assert(all_input_rels.empty() || all_input_rels.size() == 1 || all_input_rels.size() == 2); std::vector input_query_plans; rel_stack.push_back(&rel); for (const auto * input_rel : all_input_rels) @@ -297,7 +296,7 @@ QueryPlanPtr SerializedPlanParser::parseOp(const substrait::Rel & rel, std::list return query_plan; } -DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) +DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPlan & query_plan) const { const Settings & settings = parser_context->queryContext()->getSettingsRef(); QueryPriorities priorities; @@ -311,12 +310,10 @@ DB::QueryPipelineBuilderPtr SerializedPlanParser::buildQueryPipeline(DB::QueryPl settings, 0); const QueryPlanOptimizationSettings optimization_settings{.optimize_plan = settings[Setting::query_plan_enable_optimizations]}; - return query_plan.buildQueryPipeline( - optimization_settings, - BuildQueryPipelineSettings{ - .actions_settings - = ExpressionActionsSettings{.can_compile_expressions = true, .min_count_to_compile_expression = 3, .compile_expressions = CompileExpressions::yes}, - .process_list_element = query_status}); + BuildQueryPipelineSettings build_settings = BuildQueryPipelineSettings::fromContext(context); + build_settings.process_list_element = query_status; + build_settings.progress_callback = nullptr; + return query_plan.buildQueryPipeline(optimization_settings, build_settings); } std::unique_ptr SerializedPlanParser::createExecutor(const std::string_view plan) @@ -325,11 +322,10 @@ std::unique_ptr SerializedPlanParser::createExecutor(const std::s return createExecutor(parse(s_plan), s_plan); } -std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) +std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const { Stopwatch stopwatch; - const Settings & settings = parser_context->queryContext()->getSettingsRef(); DB::QueryPipelineBuilderPtr builder = nullptr; try { @@ -337,7 +333,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla } catch (...) { - LOG_ERROR(getLogger("SerializedPlanParser"), "Invalid plan:\n{}", PlanUtil::explainPlan(*query_plan)); + debug::dumpPlan(*query_plan, "Invalid clickhouse plan", true); throw; } @@ -346,13 +342,7 @@ std::unique_ptr SerializedPlanParser::createExecutor(DB::QueryPla assert(root_rel.has_root()); if (root_rel.root().input().has_write()) addSinkTransform(parser_context->queryContext(), root_rel.root().input().write(), builder); - auto * logger = &Poco::Logger::get("SerializedPlanParser"); - LOG_INFO(logger, "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); - LOG_DEBUG( - logger, - "clickhouse plan [optimization={}]:\n{}", - settings[Setting::query_plan_enable_optimizations], - PlanUtil::explainPlan(*query_plan)); + LOG_INFO(getLogger("SerializedPlanParser"), "build pipeline {} ms", stopwatch.elapsedMicroseconds() / 1000.0); auto config = ExecutorConfig::loadFromContext(parser_context->queryContext()); return std::make_unique(std::move(query_plan), std::move(builder), config.dump_pipeline); @@ -370,11 +360,7 @@ NonNullableColumnsResolver::NonNullableColumnsResolver( expression_parser = std::make_unique(parser_context); } -NonNullableColumnsResolver::~NonNullableColumnsResolver() -{ -} - -// make it simple at present, if the condition contains or, return empty for both side. +// make it simple at present if the condition contains or, return empty for both side. std::set NonNullableColumnsResolver::resolve() { collected_columns.clear(); diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.h b/cpp-ch/local-engine/Parser/SerializedPlanParser.h index 201fc46b2e9c..eadc7112c266 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.h +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.h @@ -48,7 +48,7 @@ class NonNullableColumnsResolver public: explicit NonNullableColumnsResolver( const DB::Block & header_, std::shared_ptr parser_context_, const substrait::Expression & cond_rel_); - ~NonNullableColumnsResolver(); + ~NonNullableColumnsResolver() = default; // return column names std::set resolve(); @@ -68,7 +68,7 @@ class NonNullableColumnsResolver class SerializedPlanParser { private: - std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan); + std::unique_ptr createExecutor(DB::QueryPlanPtr query_plan, const substrait::Plan & s_plan) const; public: explicit SerializedPlanParser(std::shared_ptr parser_context_); @@ -76,7 +76,7 @@ class SerializedPlanParser /// visible for UT DB::QueryPlanPtr parse(const substrait::Plan & plan); std::unique_ptr createExecutor(const substrait::Plan & plan); - DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan); + DB::QueryPipelineBuilderPtr buildQueryPipeline(DB::QueryPlan & query_plan) const; /// std::unique_ptr createExecutor(const std::string_view plan); @@ -118,6 +118,7 @@ class SerializedPlanParser private: DB::QueryPlanPtr parseOp(const substrait::Rel & rel, std::list & rel_stack); + static void adjustOutput(const DB::QueryPlanPtr & query_plan, const substrait::Plan & plan); std::vector input_iters; std::vector split_infos; diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp b/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp deleted file mode 100644 index c16405eff3c8..000000000000 --- a/cpp-ch/local-engine/Parser/SubstraitParserUtils.cpp +++ /dev/null @@ -1,57 +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. - */ - - -#include "SubstraitParserUtils.h" -#include -#include - -using namespace DB; - -namespace local_engine -{ -namespace pb_util = google::protobuf::util; -void logDebugMessage(const google::protobuf::Message & message, const char * type) -{ - if (auto * logger = &Poco::Logger::get("SubstraitPlan"); logger->debug()) - { - pb_util::JsonOptions options; - std::string json; - if (auto s = MessageToJsonString(message, &json, options); !s.ok()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert {} to Json", type); - LOG_DEBUG(logger, "{}:\n{}", type, json); - } -} -std::string dumpMessage(const google::protobuf::Message & message) -{ - pb_util::JsonOptions options; - std::string json; - if (auto s = MessageToJsonString(message, &json, options); !s.ok()) - { - if (auto * logger = &Poco::Logger::get("SubstraitPlan")) - LOG_ERROR(logger, "Can not convert message to Json"); - return ""; - } - return json; -} -std::string toString(const google::protobuf::Any & any) -{ - google::protobuf::StringValue sv; - sv.ParseFromString(any.value()); - return sv.value(); -} -} \ No newline at end of file diff --git a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h index a6c252034c56..c020d96d8e76 100644 --- a/cpp-ch/local-engine/Parser/SubstraitParserUtils.h +++ b/cpp-ch/local-engine/Parser/SubstraitParserUtils.h @@ -18,6 +18,7 @@ #include #include +#include #include namespace DB::ErrorCodes @@ -67,9 +68,10 @@ Message BinaryToMessage(const std::string_view binary) return message; } -void logDebugMessage(const google::protobuf::Message & message, const char * type); - -std::string dumpMessage(const google::protobuf::Message & message); - -std::string toString(const google::protobuf::Any & any); +inline std::string toString(const google::protobuf::Any & any) +{ + google::protobuf::StringValue sv; + sv.ParseFromString(any.value()); + return sv.value(); +} } // namespace local_engine diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp index d6584267455f..e4a56194c171 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/CommonScalarFunctionParser.cpp @@ -57,7 +57,6 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Not, not, not ); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Xor, xor, xor); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Cast, cast, CAST); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(GetTimestamp, get_timestamp, parseDateTime64InJodaSyntaxOrNull); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Quarter, quarter, toQuarter); // math functions @@ -99,8 +98,8 @@ REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Unhex, unhex, unhex); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Hypot, hypot, hypot); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Sign, sign, sign); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Radians, radians, radians); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Greatest, greatest, sparkGreatest); -REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Least, least, sparkLeast); +REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Greatest, greatest, greatest); +REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Least, least, least); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Rand, rand, randCanonical); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Bin, bin, sparkBin); REGISTER_COMMON_SCALAR_FUNCTION_PARSER(Rint, rint, sparkRint); diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp index e33d5f8717d1..935aad6244bd 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayExcept.cpp @@ -15,6 +15,7 @@ * limitations under the License. */ #include +#include #include #include #include @@ -68,20 +69,19 @@ class FunctionParserArrayExcept : public FunctionParser const auto * lambda_output = toFunctionNode(lambda_actions_dag, "not", {has_in_lambda}); lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{arr2_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {arr2_not_null}, lambda_output->result_name); // Apply arrayFilter with the lambda function @@ -95,13 +95,16 @@ class FunctionParserArrayExcept : public FunctionParser const auto * arr2_is_null_node = toFunctionNode(actions_dag, "isNull", {arr2_arg}); const auto * null_array_node = addColumnToActionsDAG(actions_dag, std::make_shared(array_distinct_node->result_type), {}); - const auto * multi_if_node = toFunctionNode(actions_dag, "multiIf", { - arr1_is_null_node, - null_array_node, - arr2_is_null_node, - null_array_node, - array_distinct_node, - }); + const auto * multi_if_node = toFunctionNode( + actions_dag, + "multiIf", + { + arr1_is_null_node, + null_array_node, + arr2_is_null_node, + null_array_node, + array_distinct_node, + }); return convertNodeTypeIfNeeded(substrait_func, multi_if_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp index 7ab829585b79..1d418ba0733d 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayHighOrderFunctions.cpp @@ -218,14 +218,14 @@ class FunctionParserArraySort : public FunctionParser const auto var_expr = expr.scalar_function().arguments()[0].value(); if (!var_expr.has_literal()) return false; - auto [_, name] = LiteralParser().parse(var_expr.literal()); + auto [_, name] = LiteralParser::parse(var_expr.literal()); return var == name.safeGet(); }; auto is_int_value = [&](const substrait::Expression & expr, Int32 val) { if (!expr.has_literal()) return false; - auto [_, x] = LiteralParser().parse(expr.literal()); + auto [_, x] = LiteralParser::parse(expr.literal()); return val == x.safeGet(); }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp index b4192ed33017..9bc67111f0d6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRemove.cpp @@ -15,11 +15,11 @@ * limitations under the License. */ #include +#include #include #include #include #include - namespace DB { namespace ErrorCodes @@ -66,20 +66,19 @@ class FunctionParserArrayRemove : public FunctionParser const auto * lambda_output = if_null_in_lambda; lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{elem_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {elem_not_null}, lambda_output->result_name); /// Apply arrayFilter with the lambda function diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp index ce1ce8e16c70..e7df0250c870 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/arrayRepeat.cpp @@ -16,6 +16,7 @@ * limitations under the License. */ #include +#include #include #include #include @@ -56,7 +57,7 @@ class FunctionParserArrayRepeat : public FunctionParser const auto * const_zero_node = addColumnToActionsDAG(actions_dag, n_not_null_arg->result_type, {0}); const auto * greatest_node = toFunctionNode(actions_dag, "greatest", {n_not_null_arg, const_zero_node}); const auto * range_node = toFunctionNode(actions_dag, "range", {greatest_node}); - const auto & range_type = assert_cast(*removeNullable(range_node->result_type)); + const auto & range_type = assert_cast(*removeNullable(range_node->result_type)); // Create lambda function x -> elem ActionsDAG lambda_actions_dag; @@ -65,20 +66,19 @@ class FunctionParserArrayRepeat : public FunctionParser const auto * lambda_output = elem_in_lambda; lambda_actions_dag.getOutputs().push_back(lambda_output); lambda_actions_dag.removeUnusedActions(Names(1, lambda_output->result_name)); - - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names{elem_in_lambda->result_name}; NamesAndTypesList lambda_arguments_names_and_types; lambda_arguments_names_and_types.emplace_back(x_in_lambda->result_name, x_in_lambda->result_type); - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_arguments_names_and_types, lambda_output->result_type, - lambda_output->result_name); + lambda_output->result_name, + false); const auto * lambda_function = &actions_dag.addFunction(function_capture, {elem_arg}, lambda_output->result_name); /// Apply arrayMap with the lambda function diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp index ebd89f8fa8e8..d663052a8ea7 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/bitLength.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ -#include +#include #include +#include namespace DB { @@ -57,7 +58,7 @@ class FunctionParserBitLength : public FunctionParser const auto * const_eight_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 8); const auto * result_node = toFunctionNode(actions_dag, "multiply", {octet_length_node, const_eight_node}); - return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp index 1586870541b6..80797dedcab5 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp @@ -16,7 +16,9 @@ */ #include +#include #include +#include #include namespace DB diff --git a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp similarity index 82% rename from cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp rename to cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp index 0de3b757e5fd..4724f820099a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionDecimalDivide.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.cpp @@ -15,14 +15,9 @@ * limitations under the License. */ -#include +#include namespace local_engine { - -REGISTER_FUNCTION(SparkFunctionDecimalDivide) -{ - factory.registerFunction>(); -} - + static FunctionParserRegister register_get_timestamp; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h new file mode 100644 index 000000000000..5e32e00569f1 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/getTimestamp.h @@ -0,0 +1,106 @@ +/* + * 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. + */ + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} +} + + +namespace local_engine +{ +class FunctionParserGetTimestamp : public FunctionParser +{ +public: + explicit FunctionParserGetTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionParserGetTimestamp() override = default; + + static constexpr auto name = "get_timestamp"; + String getName() const override { return name; } + + const ActionsDAG::Node * parse( + const substrait::Expression_ScalarFunction & substrait_func, + ActionsDAG & actions_dag) const override + { + /* + spark function: get_timestamp(expr, fmt) + 1. If timeParserPolicy is LEGACY + 1) fmt has 0 'S', ch function = parseDateTime64InJodaSyntaxOrNull(substr(expr,1,length(fmt)), fmt); + 2) fmt has 'S' more than 0, make the fmt has 3 'S', ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + 2. Else ch function = parseDateTime64InJodaSyntaxOrNull(expr, fmt) + */ + auto parsed_args = parseFunctionArguments(substrait_func, actions_dag); + if (parsed_args.size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires exactly two arguments", getName()); + const auto * expr_arg = parsed_args[0]; + const auto * fmt_arg = parsed_args[1]; + + const auto & args = substrait_func.arguments(); + bool fmt_string_literal = args[1].value().has_literal(); + String fmt; + if (fmt_string_literal) + { + const auto & literal_fmt_expr = args[1].value().literal(); + fmt_string_literal = literal_fmt_expr.has_string(); + fmt = fmt_string_literal ? literal_fmt_expr.string() : ""; + } + if (!fmt_string_literal) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The second of function {} must be const String.", name); + + UInt32 s_count = std::count(fmt.begin(), fmt.end(), 'S'); + String time_parser_policy = getContext()->getSettingsRef().has(TIMER_PARSER_POLICY) ? toString(getContext()->getSettingsRef().get(TIMER_PARSER_POLICY)) : ""; + boost::to_lower(time_parser_policy); + if (time_parser_policy == "legacy") + { + if (s_count == 0) + { + const auto * index_begin_node = addColumnToActionsDAG(actions_dag, std::make_shared(), 1); + const auto * index_end_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt.size()); + const auto * substr_node = toFunctionNode(actions_dag, "substringUTF8", {expr_arg, index_begin_node, index_end_node}); + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {substr_node, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else if (s_count < 3) + fmt += String(3 - s_count, 'S'); + else + fmt = fmt.substr(0, fmt.size() - (s_count - 3)); + + const auto * fmt_node = addColumnToActionsDAG(actions_dag, std::make_shared(), fmt); + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_node}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + else + { + const auto * result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg}); + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); + } + } +}; +} diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp index 77c0a4725684..19e3bfca056f 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/lambdaFunction.cpp @@ -17,10 +17,9 @@ #include #include #include +#include #include #include -#include -#include #include #include #include @@ -29,7 +28,7 @@ namespace DB::ErrorCodes { - extern const int LOGICAL_ERROR; +extern const int LOGICAL_ERROR; } namespace local_engine @@ -44,12 +43,10 @@ DB::NamesAndTypesList collectLambdaArguments(ParserContextPtr parser_context_, c if (arg.value().has_scalar_function() && parser_context_->getFunctionNameInSignature(arg.value().scalar_function().function_reference()) == "namedlambdavariable") { - auto [_, col_name_field] = LiteralParser().parse(arg.value().scalar_function().arguments()[0].value().literal()); + auto [_, col_name_field] = LiteralParser::parse(arg.value().scalar_function().arguments()[0].value().literal()); String col_name = col_name_field.safeGet(); if (collected_names.contains(col_name)) - { continue; - } collected_names.insert(col_name); auto type = TypeParser::parseType(arg.value().scalar_function().output_type()); lambda_arguments.emplace_back(col_name, type); @@ -63,7 +60,7 @@ class FunctionParserLambda : public FunctionParser { public: static constexpr auto name = "lambdafunction"; - explicit FunctionParserLambda(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit FunctionParserLambda(ParserContextPtr parser_context_) : FunctionParser(parser_context_) { } ~FunctionParserLambda() override = default; String getName() const override { return name; } @@ -73,15 +70,14 @@ class FunctionParserLambda : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for LambdaFunction"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { /// Some special cases, for example, `transform(arr, x -> concat(arr, array(x)))` refers to /// a column `arr` out of it directly. We need a `arr` as an input column for `lambda_actions_dag` DB::NamesAndTypesList parent_header; for (const auto * output_node : actions_dag.getOutputs()) - { parent_header.emplace_back(output_node->result_name, output_node->result_type); - } ActionsDAG lambda_actions_dag{parent_header}; /// The first argument is the lambda function body, followings are the lambda arguments which is @@ -90,28 +86,21 @@ class FunctionParserLambda : public FunctionParser /// this outside lambda function's arguments. For an example, transform(number, x -> transform(letter, y -> struct(x, y))). /// Before parsing the lambda function body, we add lambda function arguments int actions dag at first. for (size_t i = 1; i < substrait_func.arguments().size(); ++i) - { (void)parseExpression(lambda_actions_dag, substrait_func.arguments()[i].value()); - } const auto & substrait_lambda_body = substrait_func.arguments()[0].value(); const auto * lambda_body_node = parseExpression(lambda_actions_dag, substrait_lambda_body); lambda_actions_dag.getOutputs().push_back(lambda_body_node); lambda_actions_dag.removeUnusedActions(Names(1, lambda_body_node->result_name)); - auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); - auto lambda_actions = std::make_shared(std::move(lambda_actions_dag), expression_actions_settings); - DB::Names captured_column_names; - DB::Names required_column_names = lambda_actions->getRequiredColumns(); + DB::Names required_column_names = lambda_actions_dag.getRequiredColumnsNames(); DB::ActionsDAG::NodeRawConstPtrs lambda_children; auto lambda_function_args = collectLambdaArguments(parser_context, substrait_func); - const auto & lambda_actions_inputs = lambda_actions->getActionsDAG().getInputs(); + const auto & lambda_actions_inputs = lambda_actions_dag.getInputs(); std::unordered_map parent_nodes; for (const auto & node : actions_dag.getNodes()) - { parent_nodes[node.result_name] = &node; - } for (const auto & required_column_name : required_column_names) { if (std::find_if( @@ -125,13 +114,13 @@ class FunctionParserLambda : public FunctionParser lambda_actions_inputs.end(), [&required_column_name](const auto & node) { return node->result_name == required_column_name; }); if (it == lambda_actions_inputs.end()) - { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Required column not found: {}", required_column_name); - } auto parent_node_it = parent_nodes.find(required_column_name); if (parent_node_it == parent_nodes.end()) { - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Not found column {} in actions dag:\n{}", + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Not found column {} in actions dag:\n{}", required_column_name, actions_dag.dumpDAG()); } @@ -141,21 +130,23 @@ class FunctionParserLambda : public FunctionParser captured_column_names.push_back(required_column_name); } } - + auto expression_actions_settings = DB::ExpressionActionsSettings::fromContext(getContext(), DB::CompileExpressions::yes); auto function_capture = std::make_shared( - lambda_actions, + std::move(lambda_actions_dag), + expression_actions_settings, captured_column_names, lambda_function_args, lambda_body_node->result_type, - lambda_body_node->result_name); + lambda_body_node->result_name, + false); const auto * result = &actions_dag.addFunction(function_capture, lambda_children, lambda_body_node->result_name); return result; } + protected: - DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAG & actions_dag) const override + DB::ActionsDAG::NodeRawConstPtrs + parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for LambdaFunction"); } @@ -176,7 +167,7 @@ class NamedLambdaVariable : public FunctionParser { public: static constexpr auto name = "namedlambdavariable"; - explicit NamedLambdaVariable(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit NamedLambdaVariable(ParserContextPtr parser_context_) : FunctionParser(parser_context_) { } ~NamedLambdaVariable() override = default; String getName() const override { return name; } @@ -186,7 +177,8 @@ class NamedLambdaVariable : public FunctionParser throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "getCHFunctionName is not implemented for NamedLambdaVariable"); } - const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override + const DB::ActionsDAG::Node * + parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { auto [_, col_name_field] = parseLiteral(substrait_func.arguments()[0].value().literal()); String col_name = col_name_field.safeGet(); @@ -195,15 +187,13 @@ class NamedLambdaVariable : public FunctionParser const auto & inputs = actions_dag.getInputs(); auto it = std::find_if(inputs.begin(), inputs.end(), [&col_name](const auto * node) { return node->result_name == col_name; }); if (it == inputs.end()) - { return &(actions_dag.addInput(col_name, type)); - } return *it; } + protected: - DB::ActionsDAG::NodeRawConstPtrs parseFunctionArguments( - const substrait::Expression_ScalarFunction & substrait_func, - DB::ActionsDAG & actions_dag) const override + DB::ActionsDAG::NodeRawConstPtrs + parseFunctionArguments(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "parseFunctionArguments is not implemented for NamedLambdaVariable"); } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp index 34a7348b9ac6..b4e4ad119fdc 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/length.cpp @@ -15,9 +15,10 @@ * limitations under the License. */ -#include +#include #include #include +#include namespace DB { @@ -71,7 +72,7 @@ class FunctionParserLength : public FunctionParser else result_node = toFunctionNode(actions_dag, "char_length", {new_arg}); - return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, result_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp index c4f0234957af..a9ad4d9aee4a 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/locate.cpp @@ -15,10 +15,11 @@ * limitations under the License. */ +#include +#include #include #include #include -#include namespace DB { diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp index 1dd61587e011..d57d8d661b50 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/octetLength.cpp @@ -15,8 +15,9 @@ * limitations under the License. */ -#include +#include #include +#include namespace DB { @@ -52,7 +53,7 @@ class FunctionParserOctetLength : public FunctionParser new_arg = toFunctionNode(actions_dag, "CAST", {arg, string_type_node}); } const auto * octet_length_node = toFunctionNode(actions_dag, "octet_length", {new_arg}); - return convertNodeTypeIfNeeded(substrait_func, octet_length_node, actions_dag);; + return convertNodeTypeIfNeeded(substrait_func, octet_length_node, actions_dag); } }; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp index ed17c27eade9..3ffd64decb92 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/split.cpp @@ -19,14 +19,14 @@ namespace local_engine { -class SparkFunctionSplitParser : public FunctionParser +class FunctionSplitParser : public FunctionParser { public: - SparkFunctionSplitParser(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} - ~SparkFunctionSplitParser() override = default; + FunctionSplitParser(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + ~FunctionSplitParser() override = default; static constexpr auto name = "split"; String getName() const override { return name; } - String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "splitByRegexp"; } + String getCHFunctionName(const substrait::Expression_ScalarFunction &) const override { return "splitByRegexpSpark"; } const DB::ActionsDAG::Node * parse(const substrait::Expression_ScalarFunction & substrait_func, DB::ActionsDAG & actions_dag) const override { @@ -35,7 +35,7 @@ class SparkFunctionSplitParser : public FunctionParser for (const auto & arg : args) parsed_args.emplace_back(parseExpression(actions_dag, arg.value())); /// In Spark: split(str, regex [, limit] ) - /// In CH: splitByRegexp(regexp, str [, limit]) + /// In CH: splitByRegexpSpark(regexp, str [, limit]) if (parsed_args.size() >= 2) std::swap(parsed_args[0], parsed_args[1]); auto ch_function_name = getCHFunctionName(substrait_func); @@ -43,6 +43,6 @@ class SparkFunctionSplitParser : public FunctionParser return convertNodeTypeIfNeeded(substrait_func, func_node, actions_dag); } }; -static FunctionParserRegister register_split; +static FunctionParserRegister register_split; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp index 622237da9707..33997734c5e4 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/unixTimestamp.cpp @@ -17,7 +17,7 @@ #include #include - +#include namespace DB { @@ -34,10 +34,10 @@ namespace local_engine { template -class FunctionParserUnixTimestamp : public FunctionParser +class FunctionParserUnixTimestamp : public FunctionParserGetTimestamp { public: - explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParser(parser_context_) {} + explicit FunctionParserUnixTimestamp(ParserContextPtr parser_context_) : FunctionParserGetTimestamp(parser_context_) {} ~FunctionParserUnixTimestamp() override = default; static constexpr auto name = Name::name; @@ -60,13 +60,13 @@ class FunctionParserUnixTimestamp : public FunctionParser const auto * expr_arg = parsed_args[0]; const auto * fmt_arg = parsed_args[1]; auto expr_type = removeNullable(expr_arg->result_type); + if (isString(expr_type)) + return FunctionParserGetTimestamp::parse(substrait_func, actions_dag); + const DateLUTImpl * date_lut = &DateLUT::instance(); const auto * time_zone_node = addColumnToActionsDAG(actions_dag, std::make_shared(), date_lut->getTimeZone()); - const DB::ActionsDAG::Node * result_node = nullptr; - if (isString(expr_type)) - result_node = toFunctionNode(actions_dag, "parseDateTime64InJodaSyntaxOrNull", {expr_arg, fmt_arg, time_zone_node}); - else if (isDateOrDate32(expr_type)) + if (isDateOrDate32(expr_type)) result_node = toFunctionNode(actions_dag, "sparkDateToUnixTimestamp", {expr_arg, time_zone_node}); else if (isDateTime(expr_type) || isDateTime64(expr_type)) result_node = toFunctionNode(actions_dag, "toUnixTimestamp", {expr_arg, time_zone_node}); diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index aa9e15b9f036..158f1a9cfa0e 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -172,6 +173,8 @@ size_t LocalPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_write_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; @@ -342,6 +345,8 @@ size_t MemorySortLocalPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_io_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; @@ -428,6 +433,8 @@ size_t MemorySortCelebornPartitionWriter::evictPartitions() split_result->total_compress_time += compressed_output.getCompressTime(); split_result->total_io_time += compressed_output.getWriteTime(); split_result->total_serialize_time += serialization_time_watch.elapsedNanoseconds(); + compressed_output.finalize(); + output.finalize(); }; Stopwatch spill_time_watch; diff --git a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp index 02baa4a9c09c..ab4cfc18c89d 100644 --- a/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp +++ b/cpp-ch/local-engine/Shuffle/SelectorBuilder.cpp @@ -17,6 +17,7 @@ #include "SelectorBuilder.h" #include #include +#include #include #include #include diff --git a/cpp-ch/local-engine/Shuffle/ShuffleReader.h b/cpp-ch/local-engine/Shuffle/ShuffleReader.h index 3281a3c844f3..721a157849a7 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleReader.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleReader.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp index 8aa624ff9979..7167dabfad55 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp @@ -53,23 +53,20 @@ void ShuffleWriter::write(const Block & block) native_writer->write(block); } } -void ShuffleWriter::flush() +void ShuffleWriter::flush() const { if (native_writer) - { native_writer->flush(); - } } + ShuffleWriter::~ShuffleWriter() { if (native_writer) - { native_writer->flush(); - if (compression_enable) - { - compressed_out->finalize(); - } - write_buffer->finalize(); - } + + if (compression_enable) + compressed_out->finalize(); + + write_buffer->finalize(); } } diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h index 541e93e0347c..94886210c1d2 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h @@ -27,7 +27,7 @@ class ShuffleWriter jobject output_stream, jbyteArray buffer, const std::string & codecStr, jint level, bool enable_compression, size_t customize_buffer_size); virtual ~ShuffleWriter(); void write(const DB::Block & block); - void flush(); + void flush() const; private: std::unique_ptr compressed_out; diff --git a/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp b/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp index a78d615be62b..c40b474e7a8b 100644 --- a/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp +++ b/cpp-ch/local-engine/Shuffle/SparkExchangeSink.cpp @@ -16,15 +16,15 @@ */ #include "SparkExchangeSink.h" +#include #include +#include #include -#include +#include +#include #include +#include #include -#include -#include -#include -#include namespace DB @@ -74,7 +74,7 @@ void SparkExchangeSink::consume(Chunk chunk) void SparkExchangeSink::onFinish() { Stopwatch wall_time; - if (!dynamic_cast(partition_writer.get())) + if (!dynamic_cast(partition_writer.get())) { partition_writer->evictPartitions(); } @@ -222,8 +222,7 @@ void SparkExchangeManager::finish() std::vector extra_datas; for (const auto & writer : partition_writers) { - LocalPartitionWriter * local_partition_writer = dynamic_cast(writer.get()); - if (local_partition_writer) + if (LocalPartitionWriter * local_partition_writer = dynamic_cast(writer.get())) { extra_datas.emplace_back(local_partition_writer->getExtraData()); } @@ -232,12 +231,13 @@ void SparkExchangeManager::finish() chassert(extra_datas.size() == partition_writers.size()); WriteBufferFromFile output(options.data_file, options.io_buffer_size); split_result.partition_lengths = mergeSpills(output, infos, extra_datas); + output.finalize(); } split_result.wall_time += wall_time.elapsedNanoseconds(); } -void checkPartitionLengths(const std::vector & partition_length,size_t partition_num) +void checkPartitionLengths(const std::vector & partition_length, size_t partition_num) { if (partition_num != partition_length.size()) { @@ -284,7 +284,7 @@ void SparkExchangeManager::mergeSplitResult() std::vector SparkExchangeManager::gatherAllSpillInfo() const { std::vector res; - for (const auto& writer : partition_writers) + for (const auto & writer : partition_writers) { if (Spillable * spillable = dynamic_cast(writer.get())) { diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp index e2ba48e9d272..3218db2741d4 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.cpp @@ -88,7 +88,9 @@ Task CacheManager::cachePart( job_context.table.parts.clear(); job_context.table.parts.push_back(part); job_context.table.snapshot_id = ""; - Task task = [job_detail = job_context, context = this->context, read_columns = columns, only_meta_cache]() + MergeTreeCacheConfig config = MergeTreeCacheConfig::loadFromContext(context); + Task task = [job_detail = job_context, context = this->context, read_columns = columns, only_meta_cache, + prefetch_data = config.enable_data_prefetch]() { try { @@ -106,6 +108,9 @@ Task CacheManager::cachePart( job_detail.table.parts.front().name); return; } + // prefetch part data + if (prefetch_data) + storage->prefetchPartDataFile({job_detail.table.parts.front().name}); auto storage_snapshot = std::make_shared(*storage, storage->getInMemoryMetadataPtr()); NamesAndTypesList names_and_types_list; diff --git a/cpp-ch/local-engine/Storages/Cache/CacheManager.h b/cpp-ch/local-engine/Storages/Cache/CacheManager.h index 8fd26d249abc..c44026ce0bbc 100644 --- a/cpp-ch/local-engine/Storages/Cache/CacheManager.h +++ b/cpp-ch/local-engine/Storages/Cache/CacheManager.h @@ -29,7 +29,7 @@ struct MergeTreePart; struct MergeTreeTableInstance; /*** - * Manage the cache of the MergeTree, mainly including meta.bin, data.bin, metadata.gluten + * Manage the cache of the MergeTree, mainly including part_data.gluten, part_meta.gluten, metadata.gluten */ class CacheManager { diff --git a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp index 48e6950e27eb..65c2ad8af9e4 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp @@ -54,6 +54,7 @@ DB::Block NativeReader::read() DB::Block result_block; if (istr.eof()) return result_block; + if (columns_parse_util.empty()) { result_block = prepareByFirstBlock(); @@ -154,6 +155,7 @@ DB::Block NativeReader::prepareByFirstBlock() { if (istr.eof()) return {}; + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); DB::Block result_block; @@ -246,10 +248,12 @@ bool NativeReader::appendNextBlock(DB::Block & result_block) { if (istr.eof()) return false; + size_t columns = 0; size_t rows = 0; readVarUInt(columns, istr); readVarUInt(rows, istr); + for (size_t i = 0; i < columns; ++i) { // Not actually read type name. @@ -259,6 +263,7 @@ bool NativeReader::appendNextBlock(DB::Block & result_block) if (!rows) [[unlikely]] continue; + auto & column_parse_util = columns_parse_util[i]; auto & column = result_block.getByPosition(i); column_parse_util.parse(istr, column.column, rows, column_parse_util); diff --git a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp index 7f09721abbb9..c19005cd13dd 100644 --- a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp +++ b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp @@ -32,6 +32,7 @@ namespace local_engine { const String NativeWriter::AGG_STATE_SUFFIX= "#optagg"; + void NativeWriter::flush() { ostr.next(); diff --git a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp index cecb6308745c..ee6930e4de51 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.cpp @@ -15,11 +15,11 @@ * limitations under the License. */ #include "MergeSparkMergeTreeTask.h" -#include #include #include #include +#include #include #include #include @@ -94,6 +94,12 @@ bool MergeSparkMergeTreeTask::executeStep() } +void MergeSparkMergeTreeTask::cancel() noexcept +{ + if (merge_task) + merge_task->cancel(); +} + void MergeSparkMergeTreeTask::prepare() { future_part = merge_mutate_entry->future_part; diff --git a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h index ac167da3fb49..60b3328f0d1b 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MergeSparkMergeTreeTask.h @@ -65,6 +65,7 @@ class MergeSparkMergeTreeTask : public IExecutableTask txn_holder = std::move(txn_holder_); txn = std::move(txn_); } + void cancel() noexcept override; private: void prepare(); @@ -116,7 +117,7 @@ class MergeSparkMergeTreeTask : public IExecutableTask using MergeSparkMergeTreeTaskPtr = std::shared_ptr; -[[ maybe_unused ]] static void executeHere(MergeSparkMergeTreeTaskPtr task) +[[maybe_unused]] static void executeHere(MergeSparkMergeTreeTaskPtr task) { while (task->executeStep()) {} } diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp index a904fd087677..84dbc3a8d3bb 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.cpp @@ -190,6 +190,7 @@ void restoreMetaData( auto item_path = part_path / item.first; auto out = metadata_disk->writeFile(item_path); out->write(item.second.data(), item.second.size()); + out->finalize(); } }; thread_pool.scheduleOrThrow(job); @@ -253,7 +254,6 @@ void saveFileStatus( std::vector mergeParts( std::vector selected_parts, - std::unordered_map & partition_values, const String & new_part_uuid, SparkStorageMergeTree & storage, const String & partition_dir, @@ -264,13 +264,8 @@ std::vector mergeParts( future_part->assign(std::move(selected_parts)); future_part->part_info = MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION; + future_part->name = partition_dir.empty() ? "" : partition_dir + "/"; - future_part->name = ""; - if(!partition_dir.empty()) - { - future_part->name = partition_dir + "/"; - extractPartitionValues(partition_dir, partition_values); - } if(!bucket_dir.empty()) { future_part->name = future_part->name + bucket_dir + "/"; @@ -294,6 +289,8 @@ std::vector mergeParts( return merged; } +/** TODO: Remove it. + * Extract partition values from partition directory, we implement it in the java */ void extractPartitionValues(const String & partition_dir, std::unordered_map & partition_values) { Poco::StringTokenizer partitions(partition_dir, "/"); diff --git a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h index 503517cc8c7e..827788a35a65 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h +++ b/cpp-ch/local-engine/Storages/MergeTree/MetaDataHelper.h @@ -34,11 +34,8 @@ void saveFileStatus( std::vector mergeParts( std::vector selected_parts, - std::unordered_map & partition_values, const String & new_part_uuid, SparkStorageMergeTree & storage, const String & partition_dir, const String & bucket_dir); - -void extractPartitionValues(const String & partition_dir, std::unordered_map & partition_values); } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp index b7c60552468d..199bd455085d 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.cpp @@ -27,9 +27,9 @@ #include #include #include -#include - #include +#include +#include using namespace DB; using namespace local_engine; @@ -137,7 +137,7 @@ void doParseMergeTreeTableString(MergeTreeTable & table, ReadBufferFromString & assertChar('\n', in); String schema; readString(schema, in); - google::protobuf::util::JsonStringToMessage(schema, &table.schema); + table.schema = JsonStringToMessage(schema); assertChar('\n', in); readString(table.order_by_key, in); assertChar('\n', in); @@ -228,7 +228,7 @@ MergeTreeTableInstance::MergeTreeTableInstance(const google::protobuf::Any & any MergeTreeTableInstance::MergeTreeTableInstance(const substrait::ReadRel::ExtensionTable & extension_table) : MergeTreeTableInstance(extension_table.detail()) { - logDebugMessage(extension_table, "merge_tree_table"); + debug::dumpMessage(extension_table, "merge_tree_table"); } SparkStorageMergeTreePtr MergeTreeTableInstance::restoreStorage(const ContextMutablePtr & context) const @@ -243,27 +243,23 @@ std::shared_ptr MergeTreeTable::buildMetaData(const return doBuildMetadata(header.getNamesAndTypesList(), context, *this); } -MergeTreeTable::MergeTreeTable(const substrait::WriteRel & write_rel) +MergeTreeTable::MergeTreeTable(const local_engine::Write & write, const substrait::NamedStruct & table_schema) { - assert(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - local_engine::Write write_opt; - named_table.advanced_extension().optimization().UnpackTo(&write_opt); - assert(write_opt.has_mergetree()); - const Write_MergeTreeWrite & write = write_opt.mergetree(); - database = write.database(); - table = write.table(); - snapshot_id = write.snapshot_id(); - schema = write_rel.table_schema(); - order_by_key = write.order_by_key(); - low_card_key = write.low_card_key(); - minmax_index_key = write.minmax_index_key(); - bf_index_key = write.bf_index_key(); - set_index_key = write.set_index_key(); - primary_key = write.primary_key(); - relative_path = write.relative_path(); - absolute_path = write.absolute_path(); // always empty, see createNativeWrite in java - table_configs.storage_policy = write.storage_policy(); + assert(write.has_mergetree()); + const Write_MergeTreeWrite & merge_tree_write = write.mergetree(); + database = merge_tree_write.database(); + table = merge_tree_write.table(); + snapshot_id = merge_tree_write.snapshot_id(); + schema = table_schema; + order_by_key = merge_tree_write.order_by_key(); + low_card_key = merge_tree_write.low_card_key(); + minmax_index_key = merge_tree_write.minmax_index_key(); + bf_index_key = merge_tree_write.bf_index_key(); + set_index_key = merge_tree_write.set_index_key(); + primary_key = merge_tree_write.primary_key(); + relative_path = merge_tree_write.relative_path(); + absolute_path = merge_tree_write.absolute_path(); // always empty, see createNativeWrite in java + table_configs.storage_policy = merge_tree_write.storage_policy(); } std::unique_ptr buildMergeTreeSettings(const MergeTreeTableSettings & config) diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h index d015e78b3471..87b2d8403be3 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeMeta.h @@ -32,6 +32,7 @@ class ReadBufferFromString; } namespace local_engine { +class Write; class SparkStorageMergeTree; using SparkStorageMergeTreePtr = std::shared_ptr; using namespace DB; @@ -79,7 +80,7 @@ struct MergeTreeTable std::shared_ptr buildMetaData(const DB::Block & header, const ContextPtr & context) const; MergeTreeTable() = default; - explicit MergeTreeTable(const substrait::WriteRel & write_rel); + MergeTreeTable(const local_engine::Write & write, const substrait::NamedStruct & table_schema); }; struct MergeTreeTableInstance : MergeTreeTable diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp index 5a66f5933163..6c9dd890d851 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.cpp @@ -62,15 +62,24 @@ void SparkMergeTreeSink::onStart() void SparkMergeTreeSink::onFinish() { sink_helper->finish(context); + if (stats_.has_value()) + (*stats_)->collectStats(sink_helper->unsafeGet(), sink_helper->write_settings.partition_settings.partition_dir); } ///// -SinkHelperPtr SparkMergeTreeSink::create( - const MergeTreeTable & merge_tree_table, const SparkMergeTreeWriteSettings & write_settings_, const DB::ContextMutablePtr & context) +SinkToStoragePtr SparkMergeTreeSink::create( + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings_, + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats) { + if (write_settings_.partition_settings.part_name_prefix.empty()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "empty part_name_prefix is not allowed."); + auto dest_storage = merge_tree_table.getStorage(context); bool isRemoteStorage = dest_storage->getStoragePolicy()->getAnyDisk()->isRemote(); bool insert_with_local_storage = !write_settings_.insert_without_local_storage; + SinkHelperPtr sink_helper; if (insert_with_local_storage && isRemoteStorage) { auto temp = merge_tree_table.copyToDefaultPolicyStorage(context); @@ -78,10 +87,11 @@ SinkHelperPtr SparkMergeTreeSink::create( &Poco::Logger::get("SparkMergeTreeWriter"), "Create temp table {} for local merge.", temp->getStorageID().getFullNameNotQuoted()); - return std::make_shared(temp, dest_storage, write_settings_); + sink_helper = std::make_shared(temp, dest_storage, write_settings_); } - - return std::make_shared(dest_storage, write_settings_, isRemoteStorage); + else + sink_helper = std::make_shared(dest_storage, write_settings_, isRemoteStorage); + return std::make_shared(sink_helper, context, stats); } SinkHelper::SinkHelper(const SparkStorageMergeTreePtr & data_, const SparkMergeTreeWriteSettings & write_settings_, bool isRemoteStorage_) @@ -121,7 +131,7 @@ void SinkHelper::doMergePartsAsync(const std::vector & for (const auto & selected_part : prepare_merge_parts) tmp_parts.emplace(selected_part->name); - // check thread group initialized in task thread + // check a thread group initialized in task thread currentThreadGroupMemoryUsage(); thread_pool.scheduleOrThrow( [this, prepare_merge_parts, thread_group = CurrentThread::getGroup()]() -> void @@ -134,10 +144,8 @@ void SinkHelper::doMergePartsAsync(const std::vector & for (const auto & prepare_merge_part : prepare_merge_parts) before_size += prepare_merge_part->getBytesOnDisk(); - std::unordered_map partition_values; const auto merged_parts = mergeParts( prepare_merge_parts, - partition_values, toString(UUIDHelpers::generateV4()), dataRef(), write_settings.partition_settings.partition_dir, diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h index 8f02d6fa4e7d..b551d86d1d0c 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeSink.h @@ -16,10 +16,13 @@ */ #pragma once +#include #include #include #include #include +#include +#include namespace local_engine { @@ -150,16 +153,82 @@ class CopyToRemoteSinkHelper : public SinkHelper } }; +class MergeTreeStats : public WriteStatsBase +{ + DB::MutableColumns columns_; + + enum ColumnIndex + { + part_name, + partition_id, + record_count, + marks_count, + size_in_bytes + }; + + static DB::Block statsHeader() + { + return makeBlockHeader( + {{STRING(), "part_name"}, + {STRING(), "partition_id"}, + {BIGINT(), "record_count"}, + {BIGINT(), "marks_count"}, + {BIGINT(), "size_in_bytes"}}); + } + + DB::Chunk final_result() override + { + size_t rows = columns_[part_name]->size(); + return DB::Chunk(std::move(columns_), rows); + } + +public: + explicit MergeTreeStats(const DB::Block & input_header_) + : WriteStatsBase(input_header_, statsHeader()), columns_(statsHeader().cloneEmptyColumns()) + { + } + + String getName() const override { return "MergeTreeStats"; } + + void collectStats(const std::deque & parts, const std::string & partition) const + { + const size_t size = parts.size() + columns_[part_name]->size(); + columns_[part_name]->reserve(size); + columns_[partition_id]->reserve(size); + + columns_[record_count]->reserve(size); + auto & countColData = static_cast &>(*columns_[record_count]).getData(); + + columns_[marks_count]->reserve(size); + auto & marksColData = static_cast &>(*columns_[marks_count]).getData(); + + columns_[size_in_bytes]->reserve(size); + auto & bytesColData = static_cast &>(*columns_[size_in_bytes]).getData(); + + for (const auto & part : parts) + { + columns_[part_name]->insertData(part->name.c_str(), part->name.size()); + columns_[partition_id]->insertData(partition.c_str(), partition.size()); + + countColData.emplace_back(part->rows_count); + marksColData.emplace_back(part->getMarksCount()); + bytesColData.emplace_back(part->getBytesOnDisk()); + } + } +}; + class SparkMergeTreeSink : public DB::SinkToStorage { public: - static SinkHelperPtr create( + using SinkStatsOption = std::optional>; + static SinkToStoragePtr create( const MergeTreeTable & merge_tree_table, const SparkMergeTreeWriteSettings & write_settings_, - const DB::ContextMutablePtr & context); + const DB::ContextMutablePtr & context, + const SinkStatsOption & stats = {}); - explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_) - : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_) + explicit SparkMergeTreeSink(const SinkHelperPtr & sink_helper_, const ContextPtr & context_, const SinkStatsOption & stats) + : SinkToStorage(sink_helper_->metadata_snapshot->getSampleBlock()), context(context_), sink_helper(sink_helper_), stats_(stats) { } ~SparkMergeTreeSink() override = default; @@ -174,8 +243,47 @@ class SparkMergeTreeSink : public DB::SinkToStorage private: ContextPtr context; SinkHelperPtr sink_helper; - + std::optional> stats_; int part_num = 1; }; + +class SparkMergeTreePartitionedFileSink final : public SparkPartitionedBaseSink +{ + const SparkMergeTreeWriteSettings write_settings_; + MergeTreeTable table; + +public: + SparkMergeTreePartitionedFileSink( + const DB::Block & input_header, + const DB::Names & partition_by, + const MergeTreeTable & merge_tree_table, + const SparkMergeTreeWriteSettings & write_settings, + const DB::ContextPtr & context, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats), write_settings_(write_settings), table(merge_tree_table) + { + } + + SinkPtr createSinkForPartition(const String & partition_id) override + { + SparkMergeTreeWriteSettings write_settings{write_settings_}; + + assert(write_settings.partition_settings.partition_dir.empty()); + assert(write_settings.partition_settings.bucket_dir.empty()); + write_settings.partition_settings.part_name_prefix + = fmt::format("{}/{}", partition_id, write_settings.partition_settings.part_name_prefix); + write_settings.partition_settings.partition_dir = partition_id; + + return SparkMergeTreeSink::create( + table, write_settings, context_->getGlobalContext(), {std::dynamic_pointer_cast(stats_)}); + } + + // TODO implement with bucket + DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) override + { + return createSinkForPartition(partition_id); + } +}; + } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp index 75472d410c97..e584b003d2c6 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.cpp @@ -26,7 +26,8 @@ namespace local_engine IMPLEMENT_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) -void SparkMergeTreeWriteSettings::load(const DB::ContextPtr & context) +SparkMergeTreeWriteSettings::SparkMergeTreeWriteSettings(const DB::ContextPtr & context) + : partition_settings(SparkMergeTreeWritePartitionSettings::get(context)) { const DB::Settings & settings = context->getSettingsRef(); merge_after_insert = settings.get(MERGETREE_MERGE_AFTER_INSERT).safeGet(); diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h index 3554a092a470..e89b2aaf5e44 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriteSettings.h @@ -20,10 +20,10 @@ namespace local_engine { -#define MERGE_TREE_WRITE_RELATED_SETTINGS(M, ALIAS, UNIQ) \ - M(String, part_name_prefix, , "The part name prefix for writing data", UNIQ) \ - M(String, partition_dir, , "The parition directory for writing data", UNIQ) \ - M(String, bucket_dir, , "The bucket directory for writing data", UNIQ) +#define MERGE_TREE_WRITE_RELATED_SETTINGS(M, ALIAS) \ + M(String, part_name_prefix, , "The part name prefix for writing data") \ + M(String, partition_dir, , "The partition directory for writing data") \ + M(String, bucket_dir, , "The bucket directory for writing data") DECLARE_GLUTEN_SETTINGS(SparkMergeTreeWritePartitionSettings, MERGE_TREE_WRITE_RELATED_SETTINGS) @@ -35,6 +35,6 @@ struct SparkMergeTreeWriteSettings size_t merge_min_size = 1024 * 1024 * 1024; size_t merge_limit_parts = 10; - void load(const DB::ContextPtr & context); + explicit SparkMergeTreeWriteSettings(const DB::ContextPtr & context); }; } \ No newline at end of file diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp index d5c119ef44ba..a8fdfff6ff75 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.cpp @@ -103,13 +103,7 @@ std::string PartInfo::toJson(const std::vector & part_infos) writer.Key("bucket_id"); writer.String(item.bucket_id.c_str()); writer.Key("partition_values"); - writer.StartObject(); - for (const auto & key_value : item.partition_values) - { - writer.Key(key_value.first.c_str()); - writer.String(key_value.second.c_str()); - } - writer.EndObject(); + writer.String(item.partition_values.c_str()); writer.EndObject(); } writer.EndArray(); @@ -117,10 +111,7 @@ std::string PartInfo::toJson(const std::vector & part_infos) } std::unique_ptr SparkMergeTreeWriter::create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWritePartitionSettings & write_settings_, - const DB::ContextMutablePtr & context, - const std::string & spark_job_id) + const MergeTreeTable & merge_tree_table, const DB::ContextMutablePtr & context, const std::string & spark_job_id) { const DB::Settings & settings = context->getSettingsRef(); const auto dest_storage = merge_tree_table.getStorage(context); @@ -130,34 +121,22 @@ std::unique_ptr SparkMergeTreeWriter::create( Chain chain; auto sink = dest_storage->write(none, metadata_snapshot, context, false); chain.addSink(sink); + const SinkHelper & sink_helper = assert_cast(*sink).sinkHelper(); + // + // auto stats = std::make_shared(header, sink_helper); + // chain.addSink(stats); + // chain.addSource(std::make_shared( header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); chain.addSource(std::make_shared( header, settings[Setting::min_insert_block_size_rows], settings[Setting::min_insert_block_size_bytes])); - std::unordered_map partition_values; - if (!write_settings_.partition_dir.empty()) - extractPartitionValues(write_settings_.partition_dir, partition_values); - return std::make_unique( - header, - assert_cast(*sink).sinkHelper(), - QueryPipeline{std::move(chain)}, - std::move(partition_values), - spark_job_id); + return std::make_unique(header, sink_helper, QueryPipeline{std::move(chain)}, spark_job_id); } SparkMergeTreeWriter::SparkMergeTreeWriter( - const DB::Block & header_, - const SinkHelper & sink_helper_, - DB::QueryPipeline && pipeline_, - std::unordered_map && partition_values_, - const std::string & spark_job_id_) - : header{header_} - , sink_helper{sink_helper_} - , pipeline{std::move(pipeline_)} - , executor{pipeline} - , partition_values{partition_values_} - , spark_job_id(spark_job_id_) + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_) + : header{header_}, sink_helper{sink_helper_}, pipeline{std::move(pipeline_)}, executor{pipeline}, spark_job_id(spark_job_id_) { } @@ -175,7 +154,8 @@ void SparkMergeTreeWriter::close() { executor.finish(); std::string result = PartInfo::toJson(getAllPartInfo()); - SparkMergeTreeWriterJNI::setCurrentTaskWriteInfo(spark_job_id, result); + if (spark_job_id != CPP_UT_JOB_ID) + SparkMergeTreeWriterJNI::setCurrentTaskWriteInfo(spark_job_id, result); } std::vector SparkMergeTreeWriter::getAllPartInfo() const @@ -191,7 +171,7 @@ std::vector SparkMergeTreeWriter::getAllPartInfo() const part->getMarksCount(), part->getBytesOnDisk(), part->rows_count, - partition_values, + sink_helper.write_settings.partition_settings.partition_dir, sink_helper.write_settings.partition_settings.bucket_dir}); } return res; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h index 58c9a6659169..806963817871 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkMergeTreeWriter.h @@ -41,7 +41,7 @@ struct PartInfo size_t mark_count; size_t disk_size; size_t row_count; - std::unordered_map partition_values; + String partition_values; String bucket_id; bool operator<(const PartInfo & rhs) const { return disk_size < rhs.disk_size; } @@ -52,29 +52,23 @@ struct PartInfo class SparkMergeTreeWriter : public NativeOutputWriter { public: - static std::unique_ptr create( - const MergeTreeTable & merge_tree_table, - const SparkMergeTreeWritePartitionSettings & write_settings_, - const DB::ContextMutablePtr & context, - const std::string & spark_job_id); + static std::unique_ptr + create(const MergeTreeTable & merge_tree_table, const DB::ContextMutablePtr & context, const std::string & spark_job_id); SparkMergeTreeWriter( - const DB::Block & header_, - const SinkHelper & sink_helper_, - DB::QueryPipeline && pipeline_, - std::unordered_map && partition_values_, - const std::string & spark_job_id_); + const DB::Block & header_, const SinkHelper & sink_helper_, DB::QueryPipeline && pipeline_, const std::string & spark_job_id_); void write(DB::Block & block) override; void close() override; + /// visible for UTs + static constexpr auto CPP_UT_JOB_ID = "__UT_JOB_ID__"; + private: DB::Block header; const SinkHelper & sink_helper; DB::QueryPipeline pipeline; DB::PushingPipelineExecutor executor; - std::unordered_map partition_values; - const std::string spark_job_id; std::vector getAllPartInfo() const; diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp index 45be9dcf7442..c1a95a6be02c 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.cpp @@ -16,6 +16,8 @@ */ #include "SparkStorageMergeTree.h" +#include +#include #include #include #include @@ -73,7 +75,7 @@ void SparkStorageMergeTree::analysisPartsByRanges(DB::ReadFromMergeTree & source sum_ranges += part.ranges.size(); sum_marks += part.getMarksCount(); sum_rows += part.getRowsCount(); - total_marks_pk += part.data_part->index_granularity.getMarksCountWithoutFinal(); + total_marks_pk += part.data_part->index_granularity->getMarksCountWithoutFinal(); for (auto range : part.ranges) sum_marks_pk += range.getNumberOfMarks(); @@ -159,27 +161,36 @@ SparkStorageMergeTree::SparkStorageMergeTree( std::atomic SparkStorageMergeTree::part_num; -void SparkStorageMergeTree::prefetchMetaDataFile(std::unordered_set parts) const +void SparkStorageMergeTree::prefetchPartDataFile(const std::unordered_set & parts) const +{ + prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_DATA_FILE_NAME); +} + +void SparkStorageMergeTree::prefetchPartFiles(const std::unordered_set & parts, String file_name) const { auto disk = getDisks().front(); if (!disk->isRemote()) return; - std::vector meta_paths; - std::ranges::for_each(parts, [&](const String & name) { meta_paths.emplace_back(fs::path(relative_data_path) / name / "meta.bin"); }); + std::vector data_paths; + std::ranges::for_each(parts, [&](const String & name) { data_paths.emplace_back(fs::path(relative_data_path) / name / file_name); }); auto read_settings = ReadSettings{}; - // read_settings.enable_filesystem_cache = false; read_settings.remote_fs_method = RemoteFSReadMethod::read; - for (const auto & meta_path : meta_paths) + for (const auto & data_path : data_paths) { - if (!disk->existsDirectory(meta_path)) + if (!disk->existsFile(data_path)) continue; - - auto in = disk->readFile(meta_path, read_settings); + LOG_DEBUG(log, "Prefetching part file {}", data_path); + auto in = disk->readFile(data_path, read_settings); String ignore_data; readStringUntilEOF(ignore_data, *in); } } +void SparkStorageMergeTree::prefetchMetaDataFile(const std::unordered_set & parts) const +{ + prefetchPartFiles(parts, CompactObjectStorageDiskTransaction::PART_META_FILE_NAME); +} + std::vector SparkStorageMergeTree::loadDataPartsWithNames(const std::unordered_set & parts) { Stopwatch watch; @@ -214,7 +225,10 @@ MergeTreeData::LoadPartResult SparkStorageMergeTree::loadDataPart( try { - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name).withPartInfo(part_info).withPartFormatFromDisk().build(); + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getContext()->getReadSettings()) + .withPartInfo(part_info) + .withPartFormatFromDisk() + .build(); } catch (...) { @@ -429,7 +443,7 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = std::make_shared(volume->getName(), volume->getDisk(), volume->max_data_part_size); - auto new_data_part = data.getDataPartBuilder(part_dir, data_part_volume, part_dir) + auto new_data_part = data.getDataPartBuilder(part_dir, data_part_volume, part_dir, context->getReadSettings()) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) .build(); @@ -474,6 +488,12 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); auto txn = context->getCurrentTransaction(); + auto index_granularity_ptr = createMergeTreeIndexGranularity( + block.rows(), + block.bytes(), + *data.getSettings(), + new_data_part->index_granularity_info, + /*blocks_are_granules=*/false); auto out = std::make_unique( new_data_part, metadata_snapshot, @@ -481,8 +501,9 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( indices, MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()), compression_codec, + index_granularity_ptr, txn ? txn->tid : Tx::PrehistoricTID, - false, + block.bytes(), false, false, context->getWriteSettings()); @@ -500,15 +521,11 @@ MergeTreeDataWriter::TemporaryPart SparkMergeTreeDataWriter::writeTempPart( SinkToStoragePtr SparkWriteStorageMergeTree::write( const ASTPtr &, const StorageMetadataPtr & /*storage_in_memory_metadata*/, ContextPtr context, bool /*async_insert*/) { - SparkMergeTreeWriteSettings settings{.partition_settings{SparkMergeTreeWritePartitionSettings::get(context)}}; - settings.load(context); - SinkHelperPtr sink_helper = SparkMergeTreeSink::create(table, settings, getContext()); #ifndef NDEBUG auto dest_storage = table.getStorage(getContext()); assert(dest_storage.get() == this); #endif - - return std::make_shared(sink_helper, context); + return SparkMergeTreeSink::create(table, SparkMergeTreeWriteSettings{context}, getContext()); } } diff --git a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h index cec1597eab08..237cf6919208 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h +++ b/cpp-ch/local-engine/Storages/MergeTree/SparkStorageMergeTree.h @@ -71,6 +71,7 @@ class SparkStorageMergeTree : public MergeTreeData std::map getUnfinishedMutationCommands() const override; std::vector loadDataPartsWithNames(const std::unordered_set & parts); void removePartFromMemory(const MergeTreeData::DataPart & part_to_detach); + void prefetchPartDataFile(const std::unordered_set& parts) const; MergeTreeDataSelectExecutor reader; MergeTreeDataMergerMutator merger_mutator; @@ -91,7 +92,8 @@ class SparkStorageMergeTree : public MergeTreeData static std::atomic part_num; SimpleIncrement increment; - void prefetchMetaDataFile(std::unordered_set parts) const; + void prefetchPartFiles(const std::unordered_set& parts, String file_name) const; + void prefetchMetaDataFile(const std::unordered_set& parts) const; void startBackgroundMovesIfNeeded() override; std::unique_ptr getDefaultSettings() const override; LoadPartResult loadDataPart( diff --git a/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h index 2c76fe51eda4..f6bc6cda4f81 100644 --- a/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h +++ b/cpp-ch/local-engine/Storages/MergeTree/StorageMergeTreeFactory.h @@ -30,8 +30,8 @@ class DataPartStorageHolder { public: DataPartStorageHolder(const DataPartPtr& data_part, const SparkStorageMergeTreePtr& storage) - : data_part_(data_part), - storage_(storage) + : storage_(storage), + data_part_(data_part) { } @@ -48,12 +48,11 @@ class DataPartStorageHolder ~DataPartStorageHolder() { storage_->removePartFromMemory(*data_part_); - // std::cerr << fmt::format("clean part {}", data_part_->name) << std::endl; } private: - DataPartPtr data_part_; SparkStorageMergeTreePtr storage_; + DataPartPtr data_part_; }; using DataPartStorageHolderPtr = std::shared_ptr; diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp index 8c904ab20539..2d70380a8959 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.cpp @@ -18,12 +18,136 @@ #include #include +#include +#include +#include +#include namespace local_engine { +using namespace DB; + const std::string SubstraitFileSink::NO_PARTITION_ID{"__NO_PARTITION_ID__"}; -const std::string SubstraitPartitionedFileSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +const std::string SparkPartitionedBaseSink::DEFAULT_PARTITION_NAME{"__HIVE_DEFAULT_PARTITION__"}; +const std::string SparkPartitionedBaseSink::BUCKET_COLUMN_NAME{"__bucket_value__"}; +const std::vector FileNameGenerator::SUPPORT_PLACEHOLDERS{"{id}", "{bucket}"}; + +/// For Nullable(Map(K, V)) or Nullable(Array(T)), if the i-th row is null, we must make sure its nested data is empty. +/// It is for ORC/Parquet writing compatiability. For more details, refer to +/// https://github.com/apache/incubator-gluten/issues/8022 and https://github.com/apache/incubator-gluten/issues/8021 +static ColumnPtr truncateNestedDataIfNull(const ColumnPtr & column) +{ + if (const auto * col_const = checkAndGetColumn(column.get())) + { + size_t s = col_const->size(); + auto new_data = truncateNestedDataIfNull(col_const->getDataColumnPtr()); + return ColumnConst::create(std::move(new_data), s); + } + else if (const auto * col_array = checkAndGetColumn(column.get())) + { + auto new_data = truncateNestedDataIfNull(col_array->getDataPtr()); + return ColumnArray::create(std::move(new_data), col_array->getOffsetsPtr()); + } + else if (const auto * col_map = checkAndGetColumn(column.get())) + { + auto new_nested = truncateNestedDataIfNull(col_map->getNestedColumnPtr()); + return ColumnMap::create(std::move(new_nested)); + } + else if (const auto * col_tuple = checkAndGetColumn(column.get())) + { + Columns new_columns; + for (size_t i = 0; i < col_tuple->tupleSize(); ++i) + new_columns.emplace_back(truncateNestedDataIfNull(col_tuple->getColumnPtr(i))); + return ColumnTuple::create(std::move(new_columns)); + } + else if (const auto * col_nullable = checkAndGetColumn(column.get())) + { + const auto & null_map = col_nullable->getNullMapData(); + auto nested = truncateNestedDataIfNull(col_nullable->getNestedColumnPtr()); + const auto * nested_array = checkAndGetColumn(nested.get()); + const auto * nested_map = checkAndGetColumn(nested.get()); + const auto * nested_tuple = checkAndGetColumn(nested.get()); + + if (!memoryIsZero(null_map.data(), 0, null_map.size()) && (nested_array || nested_map || nested_tuple)) + { + /// Process Nullable(Array) or Nullable(Map) + if (nested_array || nested_map) + { + if (!nested_array) + nested_array = checkAndGetColumn(&nested_map->getNestedColumn()); + + const auto & offsets = nested_array->getOffsets(); + size_t total_data_size = 0; + for (size_t i = 0; i < null_map.size(); ++i) + total_data_size += (offsets[i] - offsets[i - 1]) * (!null_map[i]); + + auto new_nested_array = nested_array->cloneEmpty(); + new_nested_array->reserve(nested_array->size()); + auto & new_nested_array_data = assert_cast(*new_nested_array).getData(); + new_nested_array_data.reserve(total_data_size); + + for (size_t i = 0; i < null_map.size(); ++i) + if (null_map[i]) + new_nested_array->insertDefault(); + else + new_nested_array->insertFrom(*nested_array, i); + + if (nested_map) + { + auto new_nested_map = ColumnMap::create(std::move(new_nested_array)); + return ColumnNullable::create(std::move(new_nested_map), col_nullable->getNullMapColumnPtr()); + } + else + { + return ColumnNullable::create(std::move(new_nested_array), col_nullable->getNullMapColumnPtr()); + } + } + else + { + /// Process Nullable(Tuple) + const auto & nested_columns = nested_tuple->getColumns(); + Columns new_nested_columns(nested_columns.size()); + for (size_t i = 0; i < nested_columns.size(); ++i) + { + const auto & nested_column = nested_columns[i]; + TypeIndex type_index = nested_column->getDataType(); + if (const auto * nullable_nested_column = checkAndGetColumn(nested_column.get())) + type_index = nullable_nested_column->getNestedColumnPtr()->getDataType(); + + bool should_truncate = type_index == TypeIndex::Array || type_index == TypeIndex::Map || type_index == TypeIndex::Tuple; + if (should_truncate) + { + auto new_nested_column = nested_column->cloneEmpty(); + new_nested_column->reserve(nested_column->size()); + for (size_t j = 0; j < null_map.size(); ++j) + { + if (null_map[j]) + new_nested_column->insertDefault(); + else + new_nested_column->insertFrom(*nested_column, j); + } + new_nested_columns[i] = std::move(new_nested_column); + } + else + { + new_nested_columns[i] = nested_column; + } + } + + auto new_nested_tuple = ColumnTuple::create(std::move(new_nested_columns)); + return ColumnNullable::create(std::move(new_nested_tuple), col_nullable->getNullMapColumnPtr()); + } + } + else + { + auto new_nested = truncateNestedDataIfNull(nested); + return ColumnNullable::create(std::move(new_nested), col_nullable->getNullMapColumnPtr()); + } + } + else + return column; +} NormalFileWriter::NormalFileWriter(const OutputFormatFilePtr & file_, const DB::ContextPtr & context_) : file(file_), context(context_) { @@ -46,10 +170,12 @@ void NormalFileWriter::write(DB::Block & block) const auto & preferred_schema = file->getPreferredSchema(); for (auto & column : block) { - if (column.name.starts_with("__bucket_value__")) + if (column.name.starts_with(SparkPartitionedBaseSink::BUCKET_COLUMN_NAME)) continue; const auto & preferred_column = preferred_schema.getByPosition(index++); + /// Make sure nested array or map data is empty when the row is null in Nullable(Map(K, V)) or Nullable(Array(T)). + column.column = truncateNestedDataIfNull(column.column); column.column = DB::castColumn(column, preferred_column.type); column.name = preferred_column.name; column.type = preferred_column.type; @@ -67,7 +193,11 @@ void NormalFileWriter::close() /// When insert into a table with empty dataset, NormalFileWriter::consume would be never called. /// So we need to skip when writer is nullptr. if (writer) + { writer->finish(); + assert(output_format); + output_format->finalizeOutput(); + } } OutputFormatFilePtr createOutputFormatFile( diff --git a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h index ba918928607c..998f8d624721 100644 --- a/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h +++ b/cpp-ch/local-engine/Storages/Output/NormalFileWriter.h @@ -61,42 +61,96 @@ class NormalFileWriter : public NativeOutputWriter OutputFormatFilePtr createOutputFormatFile( const DB::ContextPtr & context, const std::string & file_uri, const DB::Block & preferred_schema, const std::string & format_hint); -class WriteStats : public DB::ISimpleTransform +struct DeltaStats { - bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks - DB::Arena partition_keys_arena_; - std::string filename_; + size_t row_count; + std::vector min; + std::vector max; + std::vector null_count; + std::set partition_index; + + static DeltaStats create(const DB::Block & output, const DB::Names & partition) + { + size_t size = output.columns() - partition.size(); + std::set partition_index; + std::ranges::transform( + partition, + std::inserter(partition_index, partition_index.end()), + [&](const auto & name) { return output.getPositionByName(name); }); + assert(partition_index.size() == partition.size()); + return DeltaStats(size, partition_index); + } + static DB::Block statsHeader(const DB::Block & output, const DB::Names & partition, DB::ColumnsWithTypeAndName && statsHeaderBase) + { + std::set partition_index; + std::ranges::transform(partition, std::inserter(partition_index, partition_index.end()), [&](const auto & name) { return name; }); + + assert(partition_index.size() == partition.size()); - absl::flat_hash_map fiel_to_count_; + auto appendBase = [&](const std::string & prefix) + { + for (const auto & column : output.getColumnsWithTypeAndName()) + if (!partition_index.contains(column.name)) + statsHeaderBase.emplace_back(wrapNullableType(column.type), prefix + column.name); + }; + appendBase("min_"); + appendBase("max_"); + for (const auto & column : output.getColumnsWithTypeAndName()) + if (!partition_index.contains(column.name)) + statsHeaderBase.emplace_back(BIGINT(), "null_count_" + column.name); + + return makeBlockHeader(statsHeaderBase); + } - static DB::Block statsHeader() + explicit DeltaStats(size_t size, const std::set & partition_index_ = {}) + : row_count(0), min(size), max(size), null_count(size, 0), partition_index(partition_index_) { - return makeBlockHeader({{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}}); } - DB::Chunk final_result() const + void update(const DB::Chunk & chunk) { - ///TODO: improve performance - auto file_col = STRING()->createColumn(); - auto partition_col = STRING()->createColumn(); - auto countCol = BIGINT()->createColumn(); - UInt64 num_rows = 0; - for (const auto & [relative_path, rows] : fiel_to_count_) + row_count += chunk.getNumRows(); + const auto & columns = chunk.getColumns(); + assert(columns.size() == min.size() + partition_index.size()); + for (size_t i = 0, col = 0; col < columns.size(); ++col) { - if (rows == 0) + if (partition_index.contains(col)) continue; - file_col->insertData(filename_.c_str(), filename_.size()); - partition_col->insertData(relative_path.data, relative_path.size); - countCol->insert(rows); - num_rows++; - } - const DB::Columns res_columns{std::move(file_col), std::move(partition_col), std::move(countCol)}; - return DB::Chunk(res_columns, num_rows); + const auto & column = columns[col]; + Int64 null_count = 0; + if (const auto * nullable_column = typeid_cast(column.get())) + { + const auto & null_map = nullable_column->getNullMapData(); + null_count = std::ranges::count_if(null_map, [](UInt8 value) { return value != 0; }); + } + this->null_count[i] += null_count; + + DB::Field min_value, max_value; + column->getExtremes(min_value, max_value); + assert(min[i].isNull() || min_value.getType() == min[i].getType()); + assert(max[i].isNull() || max_value.getType() == max[i].getType()); + if (min[i].isNull() || min_value < min[i]) + min[i] = min_value; + if (max[i].isNull() || max_value > max[i]) + max[i] = max_value; + + ++i; + } } +}; + +class WriteStatsBase : public DB::ISimpleTransform +{ +protected: + bool all_chunks_processed_ = false; /// flag to determine if we have already processed all chunks + virtual DB::Chunk final_result() = 0; public: - explicit WriteStats(const DB::Block & input_header_) : ISimpleTransform(input_header_, statsHeader(), true) { } + WriteStatsBase(const DB::Block & input_header_, const DB::Block & output_header_) + : ISimpleTransform(input_header_, output_header_, true) + { + } Status prepare() override { @@ -110,7 +164,6 @@ class WriteStats : public DB::ISimpleTransform return ISimpleTransform::prepare(); } - String getName() const override { return "WriteStats"; } void transform(DB::Chunk & chunk) override { if (all_chunks_processed_) @@ -118,40 +171,123 @@ class WriteStats : public DB::ISimpleTransform else chunk = {}; } +}; + +class WriteStats : public WriteStatsBase +{ + DB::MutableColumns columns_; + + enum ColumnIndex + { + filename, + partition_id, + record_count + }; + +protected: + DB::Chunk final_result() override + { + size_t rows = columns_[filename]->size(); + return DB::Chunk(std::move(columns_), rows); + } + +public: + WriteStats(const DB::Block & input_header_, const DB::Block & output_header_) + : WriteStatsBase(input_header_, output_header_), columns_(output_header_.cloneEmptyColumns()) + { + } - void addFilePath(const String & patition_id, const String & filename) + static std::shared_ptr create(const DB::Block & input_header_, const DB::Names & partition) { - assert(!filename.empty()); + return std::make_shared( + input_header_, + DeltaStats::statsHeader( + input_header_, partition, {{STRING(), "filename"}, {STRING(), "partition_id"}, {BIGINT(), "record_count"}})); + } - if (filename_.empty()) - filename_ = filename; + String getName() const override { return "WriteStats"; } - assert(filename_ == filename); + void collectStats(const String & filename, const String & partition, const DeltaStats & stats) const + { + // 3 => filename, partition_id, record_count + constexpr size_t baseOffset = 3; + assert(columns_.size() == baseOffset + stats.min.size() + stats.max.size() + stats.null_count.size()); + columns_[ColumnIndex::filename]->insertData(filename.c_str(), filename.size()); + columns_[partition_id]->insertData(partition.c_str(), partition.size()); + auto & countColData = static_cast &>(*columns_[record_count]).getData(); + countColData.emplace_back(stats.row_count); + size_t columnSize = stats.min.size(); + for (int i = 0; i < columnSize; ++i) + { + size_t offset = baseOffset + i; + columns_[offset]->insert(stats.min[i]); + columns_[columnSize + offset]->insert(stats.max[i]); + auto & nullCountData = static_cast &>(*columns_[(columnSize * 2) + offset]).getData(); + nullCountData.emplace_back(stats.null_count[i]); + } + } +}; + +struct FileNameGenerator +{ + // Align with org.apache.spark.sql.execution.FileNamePlaceHolder + static const std::vector SUPPORT_PLACEHOLDERS; + // Align with placeholders above + const std::vector need_to_replace; + const std::string file_pattern; + + FileNameGenerator(const std::string & file_pattern) + : file_pattern(file_pattern), need_to_replace(compute_need_to_replace(file_pattern)) + { + } + + std::vector compute_need_to_replace(const std::string & file_pattern) + { + std::vector result; + for(const std::string& placeholder: SUPPORT_PLACEHOLDERS) + { + if (file_pattern.find(placeholder) != std::string::npos) + result.push_back(true); + else + result.push_back(false); + } + return result; + } - if (patition_id.empty()) - return; - fiel_to_count_.emplace(copyStringInArena(partition_keys_arena_, patition_id), 0); + std::string generate(const std::string & bucket = "") const + { + std::string result = file_pattern; + if (need_to_replace[0]) // {id} + result = pattern_format(SUPPORT_PLACEHOLDERS[0], toString(DB::UUIDHelpers::generateV4())); + if (need_to_replace[1]) // {bucket} + result = pattern_format(SUPPORT_PLACEHOLDERS[1], bucket); + return result; } - void collectStats(const String & file_path, size_t rows) + std::string pattern_format(const std::string & arg, const std::string & replacement) const { - if (const auto it = fiel_to_count_.find(file_path); it != fiel_to_count_.end()) + std::string format_str = file_pattern; + size_t pos = format_str.find(arg); + while (pos != std::string::npos) { - it->second += rows; - return; + format_str.replace(pos, arg.length(), replacement); + pos = format_str.find(arg, pos + arg.length()); } - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "File path {} not found in the stats map", file_path); + return format_str; } }; class SubstraitFileSink final : public DB::SinkToStorage { const std::string partition_id_; + const bool bucketed_write_; const std::string relative_path_; + OutputFormatFilePtr format_file_; OutputFormatFile::OutputFormatPtr output_format_; - std::shared_ptr stats_{nullptr}; + std::shared_ptr stats_; + DeltaStats delta_stats_; - static std::string makeFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) + static std::string makeAbsoluteFilename(const std::string & base_path, const std::string & partition_id, const std::string & relative) { if (partition_id.empty()) return fmt::format("{}/{}", base_path, relative); @@ -166,49 +302,70 @@ class SubstraitFileSink final : public DB::SinkToStorage const DB::ContextPtr & context, const std::string & base_path, const std::string & partition_id, + const bool bucketed_write, const std::string & relative, const std::string & format_hint, - const DB::Block & header) + const DB::Block & header, + const std::shared_ptr & stats, + const DeltaStats & delta_stats) : SinkToStorage(header) , partition_id_(partition_id.empty() ? NO_PARTITION_ID : partition_id) + , bucketed_write_(bucketed_write) , relative_path_(relative) - , output_format_(createOutputFormatFile(context, makeFilename(base_path, partition_id, relative), header, format_hint) - ->createOutputFormat(header)) + , format_file_(createOutputFormatFile(context, makeAbsoluteFilename(base_path, partition_id, relative), header, format_hint)) + , stats_(std::dynamic_pointer_cast(stats)) + , delta_stats_(delta_stats) { } - String getName() const override { return "SubstraitFileSink"; } - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) - { - stats_ = stats; - stats_->addFilePath(partition_id_, relative_path_); - } + String getName() const override { return "SubstraitFileSink"; } protected: void consume(DB::Chunk & chunk) override { - const size_t row_count = chunk.getNumRows(); - output_format_->output->write(materializeBlock(getHeader().cloneWithColumns(chunk.detachColumns()))); + delta_stats_.update(chunk); + if (!output_format_) [[unlikely]] + output_format_ = format_file_->createOutputFormat(); - if (stats_) - stats_->collectStats(partition_id_, row_count); + const DB::Block & input_header = getHeader(); + if (bucketed_write_) + { + chunk.erase(input_header.columns() - 1); + const DB::ColumnsWithTypeAndName & cols = input_header.getColumnsWithTypeAndName(); + DB::ColumnsWithTypeAndName without_bucket_cols(cols.begin(), cols.end() - 1); + DB::Block without_bucket_header = DB::Block(without_bucket_cols); + output_format_->output->write(materializeBlock(without_bucket_header.cloneWithColumns(chunk.detachColumns()))); + } + else + output_format_->output->write(materializeBlock(input_header.cloneWithColumns(chunk.detachColumns()))); } void onFinish() override { - output_format_->output->finalize(); - output_format_->output->flush(); - output_format_->write_buffer->finalize(); + if (output_format_) [[unlikely]] + { + output_format_->finalizeOutput(); + assert(delta_stats_.row_count > 0); + if (stats_) + stats_->collectStats(relative_path_, partition_id_, delta_stats_); + } } }; -class SubstraitPartitionedFileSink final : public DB::PartitionedSink +class SparkPartitionedBaseSink : public DB::PartitionedSink { - static const std::string DEFAULT_PARTITION_NAME; public: + static const std::string DEFAULT_PARTITION_NAME; + static const std::string BUCKET_COLUMN_NAME; + + static bool isBucketedWrite(const DB::Block & input_header) + { + return input_header.has(BUCKET_COLUMN_NAME) && + input_header.getPositionByName(BUCKET_COLUMN_NAME) == input_header.columns() - 1; + } + /// visible for UTs - static DB::ASTPtr make_partition_expression(const DB::Names & partition_columns) + static DB::ASTPtr make_partition_expression(const DB::Names & partition_columns, const DB::Block & input_header) { /// Parse the following expression into ASTs /// cancat('/col_name=', 'toString(col_name)') @@ -228,16 +385,58 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink makeASTFunction("toString", DB::ASTs{column_ast}), std::make_shared(DEFAULT_PARTITION_NAME)}; arguments.emplace_back(makeASTFunction("ifNull", std::move(if_null_args))); } + if (isBucketedWrite(input_header)) + { + DB::ASTs args {std::make_shared("%05d"), std::make_shared(BUCKET_COLUMN_NAME)}; + arguments.emplace_back(DB::makeASTFunction("printf", std::move(args))); + } + assert(!arguments.empty()); + if (arguments.size() == 1) + return arguments[0]; return DB::makeASTFunction("concat", std::move(arguments)); } -private: - const std::string base_path_; - const std::string filenmame_; + DB::SinkPtr createSinkForPartition(const String & partition_id) override + { + if (bucketed_write_) + { + std::string bucket_val = partition_id.substr(partition_id.length() - 5, 5); + std::string real_partition_id = partition_id.substr(0, partition_id.length() - 5); + return createSinkForPartition(real_partition_id, bucket_val); + } + return createSinkForPartition(partition_id, ""); + } + + virtual DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) = 0; + +protected: DB::ContextPtr context_; + std::shared_ptr stats_; + DeltaStats empty_delta_stats_; + bool bucketed_write_; + +public: + SparkPartitionedBaseSink( + const DB::ContextPtr & context, + const DB::Names & partition_by, + const DB::Block & input_header, + const std::shared_ptr & stats) + : PartitionedSink(make_partition_expression(partition_by, input_header), context, input_header) + , context_(context) + , stats_(stats) + , bucketed_write_(isBucketedWrite(input_header)) + , empty_delta_stats_(DeltaStats::create(input_header, partition_by)) + { + } +}; + +class SubstraitPartitionedFileSink final : public SparkPartitionedBaseSink +{ + const std::string base_path_; + const FileNameGenerator generator_; + const DB::Block input_header_; const DB::Block sample_block_; const std::string format_hint_; - std::shared_ptr stats_{nullptr}; public: SubstraitPartitionedFileSink( @@ -246,28 +445,28 @@ class SubstraitPartitionedFileSink final : public DB::PartitionedSink const DB::Block & input_header, const DB::Block & sample_block, const std::string & base_path, - const std::string & filename, - const std::string & format_hint) - : PartitionedSink(make_partition_expression(partition_by), context, input_header) + const FileNameGenerator & generator, + const std::string & format_hint, + const std::shared_ptr & stats) + : SparkPartitionedBaseSink(context, partition_by, input_header, stats) , base_path_(base_path) - , filenmame_(filename) - , context_(context) + , generator_(generator) , sample_block_(sample_block) + , input_header_(input_header) , format_hint_(format_hint) { } - DB::SinkPtr createSinkForPartition(const String & partition_id) override + + DB::SinkPtr createSinkForPartition(const String & partition_id, const String & bucket) override { assert(stats_); - const auto partition_path = fmt::format("{}/{}", partition_id, filenmame_); - PartitionedSink::validatePartitionKey(partition_path, true); - auto file_sink = std::make_shared(context_, base_path_, partition_id, filenmame_, format_hint_, sample_block_); - file_sink->setStats(stats_); - return file_sink; + bool bucketed_write = !bucket.empty(); + std::string filename = bucketed_write ? generator_.generate(bucket) : generator_.generate(); + const auto partition_path = fmt::format("{}/{}", partition_id, filename); + validatePartitionKey(partition_path, true); + return std::make_shared( + context_, base_path_, partition_id, bucketed_write, filename, format_hint_, sample_block_, stats_, empty_delta_stats_); } String getName() const override { return "SubstraitPartitionedFileSink"; } - - ///TODO: remove this function - void setStats(const std::shared_ptr & stats) { stats_ = stats; } }; } diff --git a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h index 2ea197cddaa0..ba9b2a3c473a 100644 --- a/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ORCOutputFormatFile.h @@ -20,8 +20,8 @@ #include "config.h" #if USE_ORC -# include -# include +#include +#include namespace local_engine { @@ -33,7 +33,6 @@ class ORCOutputFormatFile : public OutputFormatFile const std::string & file_uri_, WriteBufferBuilderPtr write_buffer_builder_, const DB::Block & preferred_schema_); - ~ORCOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; }; diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp index 194d997ddf82..d5ed4309439c 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.cpp @@ -61,7 +61,6 @@ Block OutputFormatFile::createHeaderWithPreferredSchema(const Block & header) ColumnWithTypeAndName column(preferred_column.type->createColumn(), preferred_column.type, preferred_column.name); columns.emplace_back(std::move(column)); } - assert(preferred_schema.columns() == index); return {std::move(columns)}; } diff --git a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h index 7dcffc867aa3..915f9a7e7efa 100644 --- a/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/OutputFormatFile.h @@ -16,10 +16,6 @@ */ #pragma once -#include -#include -#include - #include #include #include @@ -33,9 +29,14 @@ class OutputFormatFile public: struct OutputFormat { - public: DB::OutputFormatPtr output; std::unique_ptr write_buffer; + void finalizeOutput() const + { + output->finalize(); + output->flush(); + write_buffer->finalize(); + } }; using OutputFormatPtr = std::shared_ptr; @@ -48,8 +49,8 @@ class OutputFormatFile virtual ~OutputFormatFile() = default; virtual OutputFormatPtr createOutputFormat(const DB::Block & header_) = 0; - - virtual const DB::Block getPreferredSchema() const { return preferred_schema; } + OutputFormatPtr createOutputFormat() { return createOutputFormat(preferred_schema); } + DB::Block getPreferredSchema() const { return preferred_schema; } protected: DB::Block createHeaderWithPreferredSchema(const DB::Block & header); diff --git a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h index cc87da7da854..37b843a628ec 100644 --- a/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h +++ b/cpp-ch/local-engine/Storages/Output/ParquetOutputFormatFile.h @@ -33,7 +33,6 @@ class ParquetOutputFormatFile : public OutputFormatFile const std::string & file_uri_, const WriteBufferBuilderPtr & write_buffer_builder_, const DB::Block & preferred_schema_); - ~ParquetOutputFormatFile() override = default; OutputFormatFile::OutputFormatPtr createOutputFormat(const DB::Block & header) override; }; diff --git a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp index 6926b86a34e0..c03b1918d271 100644 --- a/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/Output/WriteBufferBuilder.cpp @@ -47,7 +47,7 @@ class LocalFileWriteBufferBuilder : public WriteBufferBuilder Poco::URI file_uri(file_uri_); const String & file_path = file_uri.getPath(); - //mkdir + // mkdir std::filesystem::path p(file_path); if (!std::filesystem::exists(p.parent_path())) std::filesystem::create_directories(p.parent_path()); @@ -78,16 +78,19 @@ class HDFSFileWriteBufferBuilder : public WriteBufferBuilder auto builder = DB::createHDFSBuilder(new_file_uri, context->getConfigRef()); auto fs = DB::createHDFSFS(builder.get()); + auto begin_of_path = new_file_uri.find('/', new_file_uri.find("//") + 2); - auto last = new_file_uri.find_last_of('/'); - auto dir = new_file_uri.substr(begin_of_path, last - begin_of_path); + auto url_without_path = new_file_uri.substr(0, begin_of_path); + + // use uri.getPath() instead of new_file_uri.substr(begin_of_path) to avoid space character uri-encoded + std::filesystem::path file_path(uri.getPath()); + auto dir = file_path.parent_path().string(); + if (hdfsCreateDirectory(fs.get(), dir.c_str())) throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Cannot create dir for {} because {}", dir, std::string(hdfsGetLastError())); - const std::string hdfs_file_path = new_file_uri.substr(begin_of_path); - const std::string hdfs_uri_without_path = new_file_uri.substr(0, begin_of_path); DB::WriteSettings write_settings; - return std::make_unique(hdfs_uri_without_path, hdfs_file_path, context->getConfigRef(), 0, write_settings); + return std::make_unique(url_without_path, file_path.string(), context->getConfigRef(), 0, write_settings); } }; #endif diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp index 6a7b7b2e29cc..8c5bce2641e8 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.cpp @@ -34,7 +34,7 @@ namespace local_engine bool readDateText(LocalDate & date, DB::ReadBuffer & buf, const DB::FormatSettings & settings) { bool is_us_style = settings.date_time_input_format == DB::FormatSettings::DateTimeInputFormat::BestEffortUS; - return readDateTextWithExcel(date, buf, is_us_style); + return readDateTextWithExcel(date, buf, is_us_style, settings); } bool readDateTime64Text( @@ -84,6 +84,20 @@ bool readDatetime64TextWithExcel( /// yyyy-MM-dd'T'HH:mm:ss.SSS'Z' /// Other will fallback to ch read. /// The whole value is in buffer. + /// + auto quick_return + = [&time_zone, &scale, &datetime64]( + UInt16 year, UInt8 month, UInt8 day, UInt8 hour, UInt8 minute, UInt8 second, DB::DateTime64::NativeType fractional) -> bool + { + if (!day) + day = 1; + + if (!checkDate(year, month, day)) + return false; + + time_t datetime = time_zone.makeDateTime(year, month, day, hour, minute, second); + return DB::DecimalUtils::tryGetDecimalFromComponents(datetime, fractional, scale, datetime64); + }; UInt16 year = 0; UInt8 month = 0; @@ -91,6 +105,7 @@ bool readDatetime64TextWithExcel( UInt8 hour = 0; UInt8 minute = 0; UInt8 second = 0; + DB::DateTime64::NativeType fractional = 0; char year_digits[std::numeric_limits::digits10]; size_t num_year_digits = readDigits(year_digits, sizeof(year_digits), buf); @@ -106,11 +121,13 @@ bool readDatetime64TextWithExcel( char month_digits[std::numeric_limits::digits10]; size_t num_month_digits = readDigits(month_digits, sizeof(month_digits), buf); - if (num_month_digits != 2) + if (num_month_digits == 1) + readDecimalNumber<1>(month, month_digits); + else if (num_month_digits == 2) + readDecimalNumber<2>(month, month_digits); + else return false; - readDecimalNumber<2>(month, month_digits); - if (*buf.position() != delimiter_after_year) // delimiter must same char return false; @@ -118,13 +135,18 @@ bool readDatetime64TextWithExcel( char day_digits[std::numeric_limits::digits10]; size_t num_day_digits = readDigits(day_digits, sizeof(day_digits), buf); - if (num_day_digits != 2) + if (num_day_digits == 1) + readDecimalNumber<1>(day, day_digits); + else if (num_day_digits == 2) + readDecimalNumber<2>(day, day_digits); + else return false; - readDecimalNumber<2>(day, day_digits); - char delimiter_after_day = *buf.position(); + if (delimiter_after_day == settings.delimiter) + return quick_return(year, month, day, hour, minute, second, fractional); + if (delimiter_after_day != ' ' && delimiter_after_day != '\'') return false; @@ -159,7 +181,6 @@ bool readDatetime64TextWithExcel( /// .SSS'Z' /// if not has quote, not allow ',' after 'ss' - DB::DateTime64::NativeType fractional = 0; bool allow_comma = (settings.delimiter == ',' && quote) || (!quote && settings.delimiter != ','); if (!buf.eof() && (*buf.position() == '.' || (allow_comma && *buf.position() == ','))) { @@ -186,17 +207,10 @@ bool readDatetime64TextWithExcel( buf.position() = buf.position() + 3; } - if (!day) - day = 1; - - if (!checkDate(year, month, day)) - return false; - - time_t datetime = time_zone.makeDateTime(year, month, day, hour, minute, second); - return DB::DecimalUtils::tryGetDecimalFromComponents(datetime, fractional, scale, datetime64); + return quick_return(year, month, day, hour, minute, second, fractional); } -inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style) +inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style, const DB::FormatSettings & settings) { if (buf.eof()) return false; @@ -268,6 +282,9 @@ inline bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool i readDecimalNumber<2>(month, first_digits); char delimiter_after_year = *buf.position(); + if (delimiter_after_year == settings.csv.delimiter) + return false; + ++buf.position(); diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h index f96b31f7049d..a5f272334e54 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelReadHelpers.h @@ -98,7 +98,7 @@ bool readDateTime64Text( const DateLUTImpl & utc_time_zone, bool quote); -bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style); +bool readDateTextWithExcel(LocalDate & date, DB::ReadBuffer & buf, bool is_us_style, const DB::FormatSettings & settings); bool readDateText(LocalDate & date, DB::ReadBuffer & buf, const DB::FormatSettings & settings); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp index fc5acc533d59..ee54a8ff5e9a 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.cpp @@ -19,10 +19,12 @@ #include #include #include +#include #include #include #include + #if USE_PARQUET #include #endif @@ -56,13 +58,18 @@ FormatFile::FormatFile( for (size_t i = 0; i < file_info.partition_columns_size(); ++i) { const auto & partition_column = file_info.partition_columns(i); + std::string unescaped_key; std::string unescaped_value; Poco::URI::decode(partition_column.key(), unescaped_key); Poco::URI::decode(partition_column.value(), unescaped_value); - auto key = std::move(unescaped_key); - partition_keys.push_back(key); - partition_values[key] = std::move(unescaped_value); + + partition_keys.push_back(unescaped_key); + partition_values[unescaped_key] = unescaped_value; + + std::string normalized_key = unescaped_key; + boost::to_lower(normalized_key); + normalized_partition_values[normalized_key] = unescaped_value; } } @@ -80,7 +87,7 @@ FormatFilePtr FormatFileUtil::createFile( DB::ContextPtr context, ReadBufferBuilderPtr read_buffer_builder, const substrait::ReadRel::LocalFiles::FileOrFiles & file) { #if USE_PARQUET - if (file.has_parquet()) + if (file.has_parquet() || (file.has_iceberg() && file.iceberg().has_parquet())) { auto config = ExecutorConfig::loadFromContext(context); return std::make_shared(context, file, read_buffer_builder, config.use_local_format); @@ -88,7 +95,7 @@ FormatFilePtr FormatFileUtil::createFile( #endif #if USE_ORC - if (file.has_orc()) + if (file.has_orc() || (file.has_iceberg() && file.iceberg().has_orc())) return std::make_shared(context, file, read_buffer_builder); #endif diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index 14b53f68429c..f93abd916b52 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -65,9 +65,11 @@ class FormatFile virtual std::optional getTotalRows() { return {}; } /// Get partition keys from file path - inline const std::vector & getFilePartitionKeys() const { return partition_keys; } + const std::vector & getFilePartitionKeys() const { return partition_keys; } - inline const std::map & getFilePartitionValues() const { return partition_values; } + const std::map & getFilePartitionValues() const { return partition_values; } + + const std::map & getFileNormalizedPartitionValues() const { return normalized_partition_values; } virtual String getURIPath() const { return file_info.uri_file(); } @@ -81,6 +83,8 @@ class FormatFile ReadBufferBuilderPtr read_buffer_builder; std::vector partition_keys; std::map partition_values; + /// partition keys are normalized to lower cases for partition column case-insensitive matching + std::map normalized_partition_values; std::shared_ptr key_condition; }; using FormatFilePtr = std::shared_ptr; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index 2681ed7c1ce7..3dd0d1127f10 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -28,10 +28,11 @@ #include #include #include -#include #include #include #include +#include +#include #include #include #include @@ -70,6 +71,9 @@ namespace Setting extern const SettingsUInt64 s3_max_redirects; extern const SettingsBool s3_disable_checksum; extern const SettingsUInt64 s3_retry_attempts; +extern const SettingsMaxThreads max_download_threads; +extern const SettingsUInt64 max_download_buffer_size; +extern const SettingsBool input_format_allow_seeks; } namespace ErrorCodes { @@ -183,7 +187,7 @@ adjustReadRangeIfNeeded(std::unique_ptr read_buffer, const s return std::move(read_buffer); /// Skip text/json files with compression. - /// TODO implement adjustFileReadPosition when compression method is bzip2 + /// When the file is compressed, its read range is adjusted in [[buildWithCompressionWrapper]] Poco::URI file_uri(file_info.uri_file()); DB::CompressionMethod compression = DB::chooseCompressionMethod(file_uri.getPath(), "auto"); if (compression != CompressionMethod::None) @@ -216,6 +220,8 @@ class LocalFileReadBufferBuilder : public ReadBufferBuilder explicit LocalFileReadBufferBuilder(DB::ContextPtr context_) : ReadBufferBuilder(context_) { } ~LocalFileReadBufferBuilder() override = default; + bool isRemote() const override { return false; } + std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) override { @@ -252,9 +258,16 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder /// Get hdfs_uri Poco::URI uri(file_info.uri_file()); auto hdfs_file_path = uri.getPath(); - std::string hdfs_uri = "hdfs://" + uri.getHost(); - if (uri.getPort()) - hdfs_uri += ":" + std::to_string(uri.getPort()); + + std::string new_file_uri = uri.toString(); + if (uri.getUserInfo().empty() && BackendInitializerUtil::spark_user.has_value()) + { + uri.setUserInfo(*BackendInitializerUtil::spark_user); + new_file_uri = uri.toString(); + } + + auto begin_of_path = new_file_uri.find('/', new_file_uri.find("//") + 2); + auto hdfs_uri = new_file_uri.substr(0, begin_of_path); std::optional file_size; std::optional modified_time; @@ -321,8 +334,11 @@ class HDFSFileReadBufferBuilder : public ReadBufferBuilder DB::StoredObjects stored_objects{DB::StoredObject{remote_path, "", *file_size}}; auto cache_creator = wrapWithCache( read_buffer_creator, read_settings, remote_path, *modified_time, *file_size); + size_t buffer_size = std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + if (*file_size > 0) + buffer_size = std::min(*file_size, buffer_size); auto cache_hdfs_read = std::make_unique( - std::move(cache_creator), stored_objects, read_settings, nullptr, /* use_external_buffer */ false); + std::move(cache_creator), stored_objects, read_settings, nullptr, /* use_external_buffer */ false, buffer_size); read_buffer = std::move(cache_hdfs_read); } @@ -406,11 +422,14 @@ class S3FileReadBufferBuilder : public ReadBufferBuilder DB::StoredObjects stored_objects{DB::StoredObject{pathKey, "", object_size}}; auto s3_impl = std::make_unique( - std::move(cache_creator), stored_objects, read_settings, /* cache_log */ nullptr, /* use_external_buffer */ true); + std::move(cache_creator), stored_objects, read_settings, /* cache_log */ nullptr, /* use_external_buffer */ true, 0); auto & pool_reader = context->getThreadPoolReader(DB::FilesystemReaderType::ASYNCHRONOUS_REMOTE_FS_READER); + size_t buffer_size = std::max(read_settings.remote_fs_buffer_size, DBMS_DEFAULT_BUFFER_SIZE); + if (object_size > 0) + buffer_size = std::min(object_size, buffer_size); auto async_reader - = std::make_unique(std::move(s3_impl), pool_reader, read_settings); + = std::make_unique(std::move(s3_impl), pool_reader, read_settings, buffer_size); if (read_settings.remote_fs_prefetch) async_reader->prefetch(Priority{}); @@ -597,21 +616,20 @@ ConcurrentLRU> S3FileReadBufferBuil class AzureBlobReadBuffer : public ReadBufferBuilder { public: - explicit AzureBlobReadBuffer(DB::ContextPtr context_) : ReadBufferBuilder(context_) { } + explicit AzureBlobReadBuffer(const DB::ContextPtr & context_) : ReadBufferBuilder(context_) { } ~AzureBlobReadBuffer() override = default; std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) override { Poco::URI file_uri(file_info.uri_file()); - std::unique_ptr read_buffer; - read_buffer = std::make_unique(getClient(), file_uri.getPath(), DB::ReadSettings(), 5, 5); - return read_buffer; + return std::make_unique(getClient(), file_uri.getPath(), DB::ReadSettings(), 5, 5); } private: - std::shared_ptr shared_client; - std::shared_ptr getClient() + std::shared_ptr shared_client; + + std::shared_ptr getClient() { if (shared_client) return shared_client; @@ -654,10 +672,6 @@ void registerReadBufferBuilders() #endif } -ReadBufferBuilder::ReadBufferBuilder(DB::ContextPtr context_) : context(context_) -{ -} - DB::ReadSettings ReadBufferBuilder::getReadSettings() const { DB::ReadSettings read_settings = context->getReadSettings(); @@ -672,6 +686,10 @@ DB::ReadSettings ReadBufferBuilder::getReadSettings() const return read_settings; } +ReadBufferBuilder::ReadBufferBuilder(const DB::ContextPtr & context_) : context(context_) +{ +} + std::unique_ptr ReadBufferBuilder::wrapWithBzip2(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) { @@ -752,7 +770,11 @@ ReadBufferBuilder::buildWithCompressionWrapper(const substrait::ReadRel::LocalFi if (compression == CompressionMethod::Bzip2) return wrapWithBzip2(std::move(in), file_info); else - return wrapReadBufferWithCompressionMethod(std::move(in), compression); + { + /// In this case we are pretty sure that current split covers the whole file because only bzip2 compression is splittable + auto parallel = wrapWithParallelIfNeeded(std::move(in), file_info); + return wrapReadBufferWithCompressionMethod(std::move(parallel), compression); + } } ReadBufferBuilder::ReadBufferCreator ReadBufferBuilder::wrapWithCache( @@ -837,6 +859,39 @@ void ReadBufferBuilder::updateCaches(const String & key, size_t last_modified_ti } } +std::unique_ptr ReadBufferBuilder::wrapWithParallelIfNeeded( + std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) +{ + /// Only use parallel downloading for text and json format because data are read serially in those formats. + if (!file_info.has_text() && !file_info.has_json()) + return std::move(in); + + const auto & settings = context->getSettingsRef(); + auto max_download_threads = settings[DB::Setting::max_download_threads]; + auto max_download_buffer_size = settings[DB::Setting::max_download_buffer_size]; + + bool parallel_read = isRemote() && max_download_threads > 1 && isBufferWithFileSize(*in); + if (!parallel_read) + return std::move(in); + + size_t file_size = getFileSizeFromReadBuffer(*in); + if (file_size < 4 * max_download_buffer_size) + return std::move(in); + + LOG_TRACE( + getLogger("ReadBufferBuilder"), + "Using ParallelReadBuffer with {} workers with chunks of {} bytes", + max_download_threads, + max_download_buffer_size); + + return wrapInParallelReadBufferIfSupported( + {std::move(in)}, + DB::threadPoolCallbackRunnerUnsafe(DB::getIOThreadPool().get(), "ParallelRead"), + max_download_threads, + max_download_buffer_size, + file_size); +} + ReadBufferBuilderFactory & ReadBufferBuilderFactory::instance() { static ReadBufferBuilderFactory instance; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h index f2c1c10a6f2c..81a4c60bfcb1 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -31,10 +32,12 @@ namespace local_engine class ReadBufferBuilder { public: - explicit ReadBufferBuilder(DB::ContextPtr context_); + explicit ReadBufferBuilder(const DB::ContextPtr & context_); virtual ~ReadBufferBuilder() = default; + virtual bool isRemote() const { return true; } + /// build a new read buffer virtual std::unique_ptr build(const substrait::ReadRel::LocalFiles::FileOrFiles & file_info) = 0; @@ -55,7 +58,11 @@ class ReadBufferBuilder size_t last_modified_time, size_t file_size); + std::unique_ptr + wrapWithParallelIfNeeded(std::unique_ptr in, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info); + DB::ReadSettings getReadSettings() const; + DB::ContextPtr context; private: diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp index ffe1d18ae785..ef350472b64c 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp @@ -17,6 +17,7 @@ #include #include +#include #include #include #include @@ -87,8 +88,8 @@ SubstraitFileSource::SubstraitFileSource( /// File partition keys are read from the file path const auto partition_keys = files[0]->getFilePartitionKeys(); for (const auto & key : partition_keys) - if (to_read_header.findByName(key)) - to_read_header.erase(key); + if (const auto * col = to_read_header.findByName(key, true)) + to_read_header.erase(col->name); } } @@ -337,14 +338,15 @@ bool ConstColumnsFileReader::pull(DB::Chunk & chunk) if (const size_t col_num = header.columns()) { res_columns.reserve(col_num); - const auto & partition_values = file->getFilePartitionValues(); + const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); for (size_t pos = 0; pos < col_num; ++pos) { - auto col_with_name_and_type = header.getByPosition(pos); - auto type = col_with_name_and_type.type; - const auto & name = col_with_name_and_type.name; - auto it = partition_values.find(name); - if (it == partition_values.end()) + const auto & column = header.getByPosition(pos); + const auto & type = column.type; + const auto & name = column.name; + + auto it = normalized_partition_values.find(boost::to_lower_copy(name)); + if (it == normalized_partition_values.end()) throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Unknow partition column : {}", name); res_columns.emplace_back(createColumn(it->second, type, to_read_rows)); @@ -377,13 +379,13 @@ bool NormalFileReader::pull(DB::Chunk & chunk) if (!rows) return false; - const auto read_columns = raw_chunk.detachColumns(); - auto columns_with_name_and_type = output_header.getColumnsWithTypeAndName(); - auto partition_values = file->getFilePartitionValues(); + auto read_columns = raw_chunk.detachColumns(); + const auto & columns = output_header.getColumnsWithTypeAndName(); + const auto & normalized_partition_values = file->getFileNormalizedPartitionValues(); DB::Columns res_columns; - res_columns.reserve(columns_with_name_and_type.size()); - for (auto & column : columns_with_name_and_type) + res_columns.reserve(columns.size()); + for (auto & column : columns) { if (to_read_header.has(column.name)) { @@ -392,12 +394,11 @@ bool NormalFileReader::pull(DB::Chunk & chunk) } else { - auto it = partition_values.find(column.name); - if (it == partition_values.end()) - { + auto it = normalized_partition_values.find(boost::to_lower_copy(column.name)); + if (it == normalized_partition_values.end()) throw DB::Exception( DB::ErrorCodes::LOGICAL_ERROR, "Not found column({}) from file({}) partition keys.", column.name, file->getURIPath()); - } + res_columns.push_back(createColumn(it->second, column.type, rows)); } } diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 8ff8a866b7ca..2ec0622311be 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -74,22 +74,14 @@ static DB::ColumnWithTypeAndName getColumnFromColumnVector(JNIEnv * /*env*/, job return block->getByPosition(column_position); } -static std::string jstring2string(JNIEnv * env, jstring jStr) +static std::string jstring2string(JNIEnv * env, jstring string) { - if (!jStr) - return ""; - - const jclass string_class = env->GetObjectClass(jStr); - const jmethodID get_bytes = env->GetMethodID(string_class, "getBytes", "(Ljava/lang/String;)[B"); - auto * const string_jbytes - = static_cast(local_engine::safeCallObjectMethod(env, jStr, get_bytes, env->NewStringUTF("UTF-8"))); - SCOPE_EXIT({ - env->DeleteLocalRef(string_jbytes); - env->DeleteLocalRef(string_class); - }); - - const auto string_jbytes_a = local_engine::getByteArrayElementsSafe(env, string_jbytes); - return {reinterpret_cast(string_jbytes_a.elems()), static_cast(string_jbytes_a.length())}; + if (string == nullptr) + return std::string(); + const char * chars = env->GetStringUTFChars(string, nullptr); + std::string ret(chars); + env->ReleaseStringUTFChars(string, chars); + return ret; } extern "C" { @@ -205,8 +197,11 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n LOCAL_ENGINE_JNI_METHOD_START const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); const std::string::size_type plan_buf_size = conf_plan_a.length(); - local_engine::BackendInitializerUtil::initBackend( - local_engine::SparkConfigs::load({reinterpret_cast(conf_plan_a.elems()), plan_buf_size}, true)); + local_engine::SparkConfigs::update( + {reinterpret_cast(conf_plan_a.elems()), plan_buf_size}, + [&](const local_engine::SparkConfigs::ConfigMap & spark_conf_map) + { local_engine::BackendInitializerUtil::initBackend(spark_conf_map); }, + true); LOCAL_ENGINE_JNI_METHOD_END(env, ) } @@ -217,25 +212,6 @@ JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_n LOCAL_ENGINE_JNI_METHOD_END(env, ) } -/// TODO: remvoe this method -JNIEXPORT void Java_org_apache_gluten_vectorized_ExpressionEvaluatorJniWrapper_injectWriteFilesTempPath( - JNIEnv * env, jclass, jbyteArray temp_path, jbyteArray filename) -{ - LOCAL_ENGINE_JNI_METHOD_START - const auto query_context = local_engine::QueryContext::instance().currentQueryContext(); - - const auto path_array = local_engine::getByteArrayElementsSafe(env, temp_path); - const auto filename_array = local_engine::getByteArrayElementsSafe(env, filename); - - local_engine::GlutenWriteSettings settings{ - .task_write_tmp_dir = {reinterpret_cast(path_array.elems()), static_cast(path_array.length())}, - .task_write_filename = {reinterpret_cast(filename_array.elems()), static_cast(filename_array.length())}, - }; - settings.set(query_context); - - LOCAL_ENGINE_JNI_METHOD_END(env, ) -} - /// Set settings for the current query. It assumes that all parameters are started with `CH_RUNTIME_SETTINGS_PREFIX` prefix, /// and the prefix is removed by java before passing to C++. JNIEXPORT void @@ -824,14 +800,14 @@ JNIEXPORT jobject Java_org_apache_gluten_vectorized_CHBlockConverterJniWrapper_c spark_row_info = converter.convertCHColumnToSparkRow(*block, mask); auto * offsets_arr = env->NewLongArray(spark_row_info->getNumRows()); - const auto * offsets_src = reinterpret_cast(spark_row_info->getOffsets().data()); + const auto * offsets_src = spark_row_info->getOffsets().data(); env->SetLongArrayRegion(offsets_arr, 0, spark_row_info->getNumRows(), offsets_src); auto * lengths_arr = env->NewLongArray(spark_row_info->getNumRows()); - const auto * lengths_src = reinterpret_cast(spark_row_info->getLengths().data()); + const auto * lengths_src = spark_row_info->getLengths().data(); env->SetLongArrayRegion(lengths_arr, 0, spark_row_info->getNumRows(), lengths_src); int64_t address = reinterpret_cast(spark_row_info->getBufferAddress()); - int64_t column_number = reinterpret_cast(spark_row_info->getNumCols()); - int64_t total_size = reinterpret_cast(spark_row_info->getTotalBytes()); + int64_t column_number = spark_row_info->getNumCols(); + int64_t total_size = spark_row_info->getTotalBytes(); jobject spark_row_info_object = env->NewObject(spark_row_info_class, spark_row_info_constructor, offsets_arr, lengths_arr, address, column_number, total_size); @@ -953,38 +929,30 @@ JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniW } JNIEXPORT jlong Java_org_apache_spark_sql_execution_datasources_CHDatasourceJniWrapper_createMergeTreeWriter( - JNIEnv * env, jobject, jstring prefix, jstring partition, jstring bucket, jbyteArray writeRel, jbyteArray conf_plan) + JNIEnv * env, jobject, jbyteArray writeRel, jbyteArray conf_plan) { LOCAL_ENGINE_JNI_METHOD_START auto query_context = local_engine::QueryContext::instance().currentQueryContext(); - // by task update new configs ( in case of dynamic config update ) + // by task update new configs (in case of dynamic config update) const auto conf_plan_a = local_engine::getByteArrayElementsSafe(env, conf_plan); local_engine::SparkConfigs::updateConfig( query_context, {reinterpret_cast(conf_plan_a.elems()), static_cast(conf_plan_a.length())}); - const auto part_dir_prefix = jstring2string(env, prefix); - const auto partition_ = jstring2string(env, partition); - const auto bucket_ = jstring2string(env, bucket); - - local_engine::SparkMergeTreeWritePartitionSettings settings{ - .part_name_prefix{part_dir_prefix}, .partition_dir{partition_}, .bucket_dir{bucket_}}; - settings.set(query_context); - const auto writeRelBytes = local_engine::getByteArrayElementsSafe(env, writeRel); substrait::WriteRel write_rel = local_engine::BinaryToMessage( {reinterpret_cast(writeRelBytes.elems()), static_cast(writeRelBytes.length())}); - local_engine::MergeTreeTable merge_tree_table(write_rel); assert(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - local_engine::Write write_opt; - named_table.advanced_extension().optimization().UnpackTo(&write_opt); - assert(write_opt.has_common()); - auto * writer - = local_engine::SparkMergeTreeWriter::create(merge_tree_table, settings, query_context, write_opt.common().job_task_attempt_id()) - .release(); - - return reinterpret_cast(writer); + local_engine::Write write; + if (!named_table.advanced_extension().optimization().UnpackTo(&write)) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to unpack write optimization with local_engine::Write."); + assert(write.has_common()); + assert(write.has_mergetree()); + local_engine::MergeTreeTable merge_tree_table(write, write_rel.table_schema()); + const std::string & id = write.common().job_task_attempt_id(); + + return reinterpret_cast(local_engine::SparkMergeTreeWriter::create(merge_tree_table, query_context, id).release()); LOCAL_ENGINE_JNI_METHOD_END(env, 0) } @@ -1058,16 +1026,15 @@ JNIEXPORT jstring Java_org_apache_spark_sql_execution_datasources_CHDatasourceJn std::vector selected_parts = local_engine::StorageMergeTreeFactory::getDataPartsByNames(temp_storage->getStorageID(), "", merge_tree_table.getPartNames()); - std::unordered_map partition_values; std::vector loaded - = local_engine::mergeParts(selected_parts, partition_values, uuid_str, *temp_storage, partition_dir, bucket_dir); + = local_engine::mergeParts(selected_parts, uuid_str, *temp_storage, partition_dir, bucket_dir); std::vector res; for (auto & partPtr : loaded) { saveFileStatus(*temp_storage, context, partPtr->name, const_cast(partPtr->getDataPartStorage())); res.emplace_back(local_engine::PartInfo{ - partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_values, bucket_dir}); + partPtr->name, partPtr->getMarksCount(), partPtr->getBytesOnDisk(), partPtr->rows_count, partition_dir, bucket_dir}); } auto json_info = local_engine::PartInfo::toJson(res); @@ -1310,10 +1277,11 @@ JNIEXPORT jlong Java_org_apache_gluten_vectorized_SimpleExpressionEval_nativeNex LOCAL_ENGINE_JNI_METHOD_END(env, -1) } -JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_createThreadGroup(JNIEnv * env, jclass) +JNIEXPORT jlong Java_org_apache_gluten_memory_CHThreadGroup_createThreadGroup(JNIEnv * env, jclass, jstring task_id_) { LOCAL_ENGINE_JNI_METHOD_START - return local_engine::QueryContext::instance().initializeQuery(); + auto task_id = jstring2string(env, task_id_); + return local_engine::QueryContext::instance().initializeQuery(task_id); LOCAL_ENGINE_JNI_METHOD_END(env, 0l) } diff --git a/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp b/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp index 4ef9b5771af8..a50bcf170eff 100644 --- a/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_cast_float_function.cpp @@ -52,7 +52,7 @@ static void BM_CHCastFloatToInt(benchmark::State & state) args.emplace_back(type_name_col); auto executable = function->build(args); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkCastFloatToInt(benchmark::State & state) @@ -63,7 +63,7 @@ static void BM_SparkCastFloatToInt(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHCastFloatToInt)->Unit(benchmark::kMillisecond)->Iterations(100); diff --git a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp index 13e74abaee1f..20ab61265e5b 100644 --- a/cpp-ch/local-engine/tests/benchmark_local_engine.cpp +++ b/cpp-ch/local-engine/tests/benchmark_local_engine.cpp @@ -846,7 +846,7 @@ QueryPlanPtr joinPlan(QueryPlanPtr left, QueryPlanPtr right, String left_key, St auto hash_join = std::make_shared(join, right->getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 1, false); + = std::make_unique(left->getCurrentHeader(), right->getCurrentHeader(), hash_join, block_size, 8192, 1, NameSet{}, false, false); std::vector plans; plans.emplace_back(std::move(left)); diff --git a/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp b/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp index ef961f21cbb6..a672fdee350a 100644 --- a/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_spark_floor_function.cpp @@ -66,7 +66,7 @@ static void BM_CHFloorFunction_For_Int64(benchmark::State & state) auto executable = function->build(int64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows()); + auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -80,7 +80,7 @@ static void BM_CHFloorFunction_For_Float64(benchmark::State & state) auto executable = function->build(float64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows()); + auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -94,7 +94,7 @@ static void BM_SparkFloorFunction_For_Int64(benchmark::State & state) auto executable = function->build(int64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows()); + auto result = executable->execute(int64_block.getColumnsWithTypeAndName(), executable->getResultType(), int64_block.rows(), false); benchmark::DoNotOptimize(result); } } @@ -108,7 +108,7 @@ static void BM_SparkFloorFunction_For_Float64(benchmark::State & state) auto executable = function->build(float64_block.getColumnsWithTypeAndName()); for (auto _ : state) { - auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows()); + auto result = executable->execute(float64_block.getColumnsWithTypeAndName(), executable->getResultType(), float64_block.rows(), false); benchmark::DoNotOptimize(result); } } diff --git a/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp b/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp index c72125163351..49f9dde989e9 100644 --- a/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_to_datetime_function.cpp @@ -45,7 +45,7 @@ static void BM_CHParseDateTime64(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } @@ -57,7 +57,7 @@ static void BM_SparkParseDateTime64(benchmark::State & state) Block block = createDataBlock(30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHParseDateTime64)->Unit(benchmark::kMillisecond)->Iterations(50); diff --git a/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp b/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp index e7abfda7a2b2..a7dc3ffa2b91 100644 --- a/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp +++ b/cpp-ch/local-engine/tests/benchmark_unix_timestamp_function.cpp @@ -49,7 +49,7 @@ static void BM_CHUnixTimestamp_For_Date32(benchmark::State & state) Block block = createDataBlock("Date32", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_CHUnixTimestamp_For_Date(benchmark::State & state) @@ -60,7 +60,7 @@ static void BM_CHUnixTimestamp_For_Date(benchmark::State & state) Block block = createDataBlock("Date", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkUnixTimestamp_For_Date32(benchmark::State & state) @@ -71,7 +71,7 @@ static void BM_SparkUnixTimestamp_For_Date32(benchmark::State & state) Block block = createDataBlock("Date32", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } static void BM_SparkUnixTimestamp_For_Date(benchmark::State & state) @@ -82,7 +82,7 @@ static void BM_SparkUnixTimestamp_For_Date(benchmark::State & state) Block block = createDataBlock("Date", 30000000); auto executable = function->build(block.getColumnsWithTypeAndName()); for (auto _ : state) [[maybe_unused]] - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); } BENCHMARK(BM_CHUnixTimestamp_For_Date32)->Unit(benchmark::kMillisecond)->Iterations(100); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.cpp b/cpp-ch/local-engine/tests/gluten_test_util.cpp index 7d0961094363..455abdb62e41 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.cpp +++ b/cpp-ch/local-engine/tests/gluten_test_util.cpp @@ -22,8 +22,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -76,10 +76,9 @@ std::optional parseFilter(const std::string & filter, const AnotherR return ActionsDAG::buildFilterActionsDAG({visitor_data.getActions().getOutputs().back()}, node_name_to_input_column); } -std::pair> create_plan_and_executor( - std::string_view json_plan, std::string_view split_template, std::string_view file, const std::optional & context) +std::pair> +create_plan_and_executor(std::string_view json_plan, std::string_view split, const std::optional & context) { - const std::string split = replaceLocalFilesWildcards(split_template, file); const auto plan = local_engine::JsonStringToMessage(json_plan); auto parser_context = ParserContext::build(context.value_or(QueryContext::globalContext()), plan); SerializedPlanParser parser(parser_context); @@ -87,6 +86,13 @@ std::pair> create_plan_and_execu return {plan, parser.createExecutor(plan)}; } +std::pair> create_plan_and_executor( + std::string_view json_plan, std::string_view split_template, std::string_view file, const std::optional & context) +{ + const std::string split = replaceLocalFilesWildcards(split_template, file); + return create_plan_and_executor(json_plan, split, context); +} + const char * get_data_dir() { const auto * const result = std::getenv("PARQUET_TEST_DATA"); diff --git a/cpp-ch/local-engine/tests/gluten_test_util.h b/cpp-ch/local-engine/tests/gluten_test_util.h index a9d8af37b47a..a61612662961 100644 --- a/cpp-ch/local-engine/tests/gluten_test_util.h +++ b/cpp-ch/local-engine/tests/gluten_test_util.h @@ -16,6 +16,7 @@ */ #pragma once +#include "testConfig.h" #include #include @@ -71,6 +72,9 @@ AnotherRowType readParquetSchema(const std::string & file); std::optional parseFilter(const std::string & filter, const AnotherRowType & name_and_types); +std::pair> +create_plan_and_executor(std::string_view json_plan, std::string_view split, const std::optional & context = std::nullopt); + std::pair> create_plan_and_executor( std::string_view json_plan, std::string_view split_template, @@ -85,6 +89,13 @@ inline std::string replaceLocalFilesWildcards(const std::string_view haystack, c return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); } +inline std::string replaceLocalFilesWithTPCH(const std::string_view haystack) +{ + static constexpr auto wildcard = "{replace_GLUTEN_SOURCE_TPCH_DIR}"; + constexpr std::string_view replaced = GLUTEN_SOURCE_TPCH_DIR(""); + return boost::replace_all_copy(std::string{haystack}, wildcard, replaced); +} + inline BlockFieldType toBlockFieldType(const AnotherFieldType & type) { return BlockFieldType(type.type, type.name); diff --git a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp index e905bc1787fa..c9bff195c5d4 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_functions.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_functions.cpp @@ -47,7 +47,7 @@ TEST(TestFuntion, Hash) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(0), result->getUInt(1)); @@ -79,7 +79,8 @@ TEST(TestFunction, In) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); @@ -89,7 +90,7 @@ TEST(TestFunction, In) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(3), 0); @@ -122,7 +123,8 @@ TEST(TestFunction, NotIn1) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); @@ -133,7 +135,7 @@ TEST(TestFunction, NotIn1) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); std::cerr << "output:\n"; debug::headColumn(result); ASSERT_EQ(result->getUInt(3), 1); @@ -165,7 +167,8 @@ TEST(TestFunction, NotIn2) set->setHeader(col1_set_block.getColumnsWithTypeAndName()); set->insertFromBlock(col1_set_block.getColumnsWithTypeAndName()); set->finishInsert(); - auto future_set = std::make_shared(std::move(set)); + PreparedSets::Hash empty; + auto future_set = std::make_shared(empty, std::move(set)); //TODO: WHY? after https://github.com/ClickHouse/ClickHouse/pull/63723 we need pass 4 instead of 1 auto arg = ColumnSet::create(4, future_set); @@ -176,14 +179,14 @@ TEST(TestFunction, NotIn2) std::cerr << "input:\n"; debug::headBlock(block); auto executable = function->build(block.getColumnsWithTypeAndName()); - auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows()); + auto result = executable->execute(block.getColumnsWithTypeAndName(), executable->getResultType(), block.rows(), false); auto function_not = factory.get("not", local_engine::QueryContext::globalContext()); auto type_bool = DataTypeFactory::instance().get("UInt8"); ColumnsWithTypeAndName columns2 = {ColumnWithTypeAndName(result, type_bool, "string0")}; Block block2(columns2); auto executable2 = function_not->build(block2.getColumnsWithTypeAndName()); - auto result2 = executable2->execute(block2.getColumnsWithTypeAndName(), executable2->getResultType(), block2.rows()); + auto result2 = executable2->execute(block2.getColumnsWithTypeAndName(), executable2->getResultType(), block2.rows(), false); std::cerr << "output:\n"; debug::headColumn(result2); ASSERT_EQ(result2->getUInt(3), 1); diff --git a/cpp-ch/local-engine/tests/gtest_ch_join.cpp b/cpp-ch/local-engine/tests/gtest_ch_join.cpp index 52120cede088..5df5eaff8c43 100644 --- a/cpp-ch/local-engine/tests/gtest_ch_join.cpp +++ b/cpp-ch/local-engine/tests/gtest_ch_join.cpp @@ -97,6 +97,10 @@ TEST(TestJoin, simple) for (const auto & column : join->columnsFromJoinedTable()) join->addJoinedColumn(column); + auto columns_from_left_table = left_plan.getCurrentHeader().getNamesAndTypesList(); + for (auto & column_from_joined_table : columns_from_left_table) + join->setUsedColumn(column_from_joined_table, JoinTableSide::Left); + auto left_keys = left.getNamesAndTypesList(); join->addJoinedColumnsAndCorrectTypes(left_keys, true); std::cerr << "after join:\n"; @@ -123,7 +127,7 @@ TEST(TestJoin, simple) auto hash_join = std::make_shared(join, right_plan.getCurrentHeader()); QueryPlanStepPtr join_step - = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 1, false); + = std::make_unique(left_plan.getCurrentHeader(), right_plan.getCurrentHeader(), hash_join, 8192, 8192, 1, NameSet{}, false, false); std::cerr << "join step:" << join_step->getOutputHeader().dumpStructure() << std::endl; @@ -145,4 +149,4 @@ TEST(TestJoin, simple) auto res = pipeline->getHeader().cloneEmpty(); executor.pull(res); debug::headBlock(res); -} \ No newline at end of file +} diff --git a/cpp-ch/local-engine/tests/gtest_local_engine.cpp b/cpp-ch/local-engine/tests/gtest_local_engine.cpp index 973f2fa6192f..06e94e051b86 100644 --- a/cpp-ch/local-engine/tests/gtest_local_engine.cpp +++ b/cpp-ch/local-engine/tests/gtest_local_engine.cpp @@ -67,6 +67,7 @@ TEST(ReadBufferFromFile, seekBackwards) WriteBufferFromFile out(tmp_file->path()); for (size_t i = 0; i < N; ++i) writeIntBinary(i, out); + out.finalize(); } ReadBufferFromFile in(tmp_file->path(), BUF_SIZE); @@ -94,8 +95,10 @@ void registerOutputFormatParquet(DB::FormatFactory & factory); int main(int argc, char ** argv) { - BackendInitializerUtil::initBackend( - SparkConfigs::load(local_engine::JsonStringToBinary(EMBEDDED_PLAN(_config_json)), true)); + SparkConfigs::update( + local_engine::JsonStringToBinary(EMBEDDED_PLAN(_config_json)), + [&](const SparkConfigs::ConfigMap & spark_conf_map) { BackendInitializerUtil::initBackend(spark_conf_map); }, + true); auto & factory = FormatFactory::instance(); DB::registerOutputFormatParquet(factory); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp index 233456992dff..288cd6f0ec6a 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -120,7 +121,7 @@ TEST(WritePipeline, SubstraitFileSink) const auto context = DB::Context::createCopy(QueryContext::globalContext()); GlutenWriteSettings settings{ .task_write_tmp_dir = "file:///tmp/test_table/test", - .task_write_filename = "data.parquet", + .task_write_filename_pattern = "data.parquet", }; settings.set(context); @@ -138,14 +139,6 @@ TEST(WritePipeline, SubstraitFileSink) EXPECT_TRUE(write_rel.has_named_table()); const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - EXPECT_EQ(2, config.size()); - EXPECT_EQ("parquet", config["format"]); - EXPECT_EQ("1", config["isSnappy"]); - EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -159,10 +152,10 @@ TEST(WritePipeline, SubstraitFileSink) EXPECT_TRUE(local_executor->hasNext()); const Block & x = *local_executor->nextColumnar(); - debug::headBlock(x); + std::cerr << debug::verticalShowString(x, 10, 50) << std::endl; EXPECT_EQ(1, x.rows()); const auto & col_a = *(x.getColumns()[0]); - EXPECT_EQ(settings.task_write_filename, col_a.getDataAt(0)); + EXPECT_EQ(settings.task_write_filename_pattern, col_a.getDataAt(0)); const auto & col_b = *(x.getColumns()[1]); EXPECT_EQ(SubstraitFileSink::NO_PARTITION_ID, col_b.getDataAt(0)); const auto & col_c = *(x.getColumns()[2]); @@ -176,7 +169,7 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const auto context = DB::Context::createCopy(QueryContext::globalContext()); GlutenWriteSettings settings{ .task_write_tmp_dir = "file:///tmp/test_table/test_partition", - .task_write_filename = "data.parquet", + .task_write_filename_pattern = "data.parquet", }; settings.set(context); @@ -193,16 +186,6 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const substrait::WriteRel & write_rel = root_rel.root().input().write(); EXPECT_TRUE(write_rel.has_named_table()); - const substrait::NamedObjectWrite & named_table = write_rel.named_table(); - - google::protobuf::StringValue optimization; - named_table.advanced_extension().optimization().UnpackTo(&optimization); - auto config = local_engine::parse_write_parameter(optimization.value()); - EXPECT_EQ(2, config.size()); - EXPECT_EQ("parquet", config["format"]); - EXPECT_EQ("1", config["isSnappy"]); - - EXPECT_TRUE(write_rel.has_table_schema()); const substrait::NamedStruct & table_schema = write_rel.table_schema(); auto block = TypeParser::buildBlockFromNamedStruct(table_schema); @@ -218,24 +201,22 @@ TEST(WritePipeline, SubstraitPartitionedFileSink) const Block & x = *local_executor->nextColumnar(); debug::headBlock(x, 25); EXPECT_EQ(25, x.rows()); - // const auto & col_b = *(x.getColumns()[1]); - // EXPECT_EQ(16, col_b.getInt(0)); } TEST(WritePipeline, ComputePartitionedExpression) { const auto context = DB::Context::createCopy(QueryContext::globalContext()); - - auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}); + + Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; + auto partition_by = SubstraitPartitionedFileSink::make_partition_expression({"s_nationkey", "name"}, sample_block); + // auto partition_by = printColumn("s_nationkey"); ASTs arguments(1, partition_by); ASTPtr partition_by_string = makeASTFunction("toString", std::move(arguments)); - Block sample_block{{STRING(), "name"}, {UINT(), "s_nationkey"}}; + auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); auto partition_by_expr = ExpressionAnalyzer(partition_by_string, syntax_result, context).getActions(false); - - auto partition_by_column_name = partition_by_string->getColumnName(); Chunk chunk = testChunk(); diff --git a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp index dbff85806147..1ad90060f475 100644 --- a/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp +++ b/cpp-ch/local-engine/tests/gtest_write_pipeline_mergetree.cpp @@ -38,6 +38,7 @@ #include #include #include +#include #include #include @@ -194,7 +195,6 @@ INCBIN(_1_read_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/1_p TEST(MergeTree, SparkMergeTree) { - GTEST_SKIP(); // TODO: fix test ThreadStatus thread_status; const auto context = DB::Context::createCopy(QueryContext::globalContext()); @@ -226,7 +226,7 @@ TEST(MergeTree, SparkMergeTree) }; gm_write_settings.set(context); - auto writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, gm_write_settings, context, "no job id"); + auto writer = local_engine::SparkMergeTreeWriter::create(merge_tree_table, context, SparkMergeTreeWriter::CPP_UT_JOB_ID); SparkMergeTreeWriter & spark_merge_tree_writer = *writer; auto [_, local_executor] = test::create_plan_and_executor(EMBEDDED_PLAN(_1_read_), split_template, file); @@ -255,25 +255,42 @@ TEST(MergeTree, SparkMergeTree) } } -INCBIN(_2_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/2_one_pipeline.json"); - -TEST(MergeTree, Pipeline) +INCBIN(_3_mergetree_plan_input_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/lineitem_parquet_input.json"); +namespace +{ +void writeMerge(std::string_view json_plan, + const std::string & outputPath , + const std::function & callback, std::optional input = std::nullopt) { - GTEST_SKIP(); const auto context = DB::Context::createCopy(QueryContext::globalContext()); - GlutenWriteSettings settings{ - .task_write_tmp_dir = "file:///tmp/lineitem_mergetree", - .task_write_filename = "part-00000-a09f9d59-2dc6-43bc-a485-dcab8384b2ff.c000.mergetree", - }; + GlutenWriteSettings settings{.task_write_tmp_dir = outputPath}; settings.set(context); + SparkMergeTreeWritePartitionSettings partition_settings{.part_name_prefix = "pipline_prefix"}; + partition_settings.set(context); - constexpr std::string_view split_template - = R"({"items":[{"uriFile":"{replace_local_files}","length":"19230111","parquet":{},"schema":{},"metadataColumns":[{}],"properties":{"fileSize":"19230111","modificationTime":"1722330598029"}}]})"; - auto [_, local_executor] = test::create_plan_and_executor( - EMBEDDED_PLAN(_2_mergetree_plan_), - split_template, - GLUTEN_SOURCE_TPCH_DIR("lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet"), - context); - EXPECT_TRUE(local_executor->hasNext()); - const Block & x = *local_executor->nextColumnar(); + auto input_json = input.value_or(replaceLocalFilesWithTPCH(EMBEDDED_PLAN(_3_mergetree_plan_input_))); + auto [_, local_executor] = test::create_plan_and_executor(json_plan, input_json, context); + + while (local_executor->hasNext()) + callback(*local_executor->nextColumnar()); +} +} +INCBIN(_3_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/3_one_pipeline.json"); +INCBIN(_4_mergetree_plan_, SOURCE_DIR "/utils/extern-local-engine/tests/json/mergetree/4_one_pipeline.json"); +TEST(MergeTree, Pipeline) +{ + writeMerge(EMBEDDED_PLAN(_3_mergetree_plan_),"tmp/lineitem_mergetree",[&](const DB::Block & block) + { + EXPECT_EQ(1, block.rows()); + debug::headBlock(block); + }); +} + +TEST(MergeTree, PipelineWithPartition) +{ + writeMerge(EMBEDDED_PLAN(_4_mergetree_plan_),"tmp/lineitem_mergetree_p",[&](const DB::Block & block) + { + EXPECT_EQ(2525, block.rows()); + debug::headBlock(block); + }); } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json index eecb8d7484d2..81f3fdf77241 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree.json @@ -1,6 +1,6 @@ { "detail": { "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "MergeTree;default\nlineitem_mergetree\n1724764750266_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\nlineitem_mergetree\n\n{\"storage_policy\":\"default\"}\n" + "value": "MergeTree;default\nlineitem_mergetree\n1724764750266_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n\nlineitem_mergetree\n\n{\"storage_policy\":\"default\"}\n" } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json index d1ed674596e5..efbd37e11ea5 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json +++ b/cpp-ch/local-engine/tests/json/mergetree/1_mergetree_hdfs.json @@ -1,6 +1,6 @@ { "detail": { "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "MergeTree;default\nlineitem_mergetree_hdfs\n1724766584676_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n3.5/test/lineitem_mergetree_hdfs\n\n{\"storage_policy\":\"__hdfs_main\"}\n" + "value": "MergeTree;default\nlineitem_mergetree_hdfs\n1724766584676_0\n{\"names\":[\"l_orderkey\",\"l_partkey\",\"l_suppkey\",\"l_linenumber\",\"l_quantity\",\"l_extendedprice\",\"l_discount\",\"l_tax\",\"l_returnflag\",\"l_linestatus\",\"l_shipdate\",\"l_commitdate\",\"l_receiptdate\",\"l_shipinstruct\",\"l_shipmode\",\"l_comment\"],\"struct\":{\"types\":[{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"i64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"fp64\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"date\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}},{\"string\":{\"nullability\":\"NULLABILITY_NULLABLE\"}}]}}\ntuple()\n\n\n\n\n\n3.5/test/lineitem_mergetree_hdfs\n\n{\"storage_policy\":\"__hdfs_main\"}\n" } } \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json similarity index 96% rename from cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json rename to cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json index fbc593267464..b1de89a63d3a 100644 --- a/cpp-ch/local-engine/tests/json/mergetree/2_one_pipeline.json +++ b/cpp-ch/local-engine/tests/json/mergetree/3_one_pipeline.json @@ -7,8 +7,17 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=mergetree\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "mergetree" + }, + "mergetree": { + "database": "default", + "table": "lineitem_mergetree", + "snapshotId": "1729651573302_0", + "orderByKey": "tuple()", + "storagePolicy": "default" + } }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", diff --git a/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json new file mode 100644 index 000000000000..14a9b3dda2ad --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/4_one_pipeline.json @@ -0,0 +1,377 @@ +{ + "relations": [ + { + "root": { + "input": { + "write": { + "namedTable": { + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "mergetree" + }, + "mergetree": { + "database": "default", + "table": "lineitem_mergetree_insertoverwrite2", + "snapshotId": "1731309448915_0", + "orderByKey": "tuple()", + "storagePolicy": "default" + } + }, + "enhancement": { + "@type": "type.googleapis.com/substrait.Type", + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ], + "nullability": "NULLABILITY_REQUIRED" + } + } + } + }, + "tableSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "PARTITION_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "input": { + "read": { + "common": { + "direct": {} + }, + "baseSchema": { + "names": [ + "l_orderkey", + "l_partkey", + "l_suppkey", + "l_linenumber", + "l_quantity", + "l_extendedprice", + "l_discount", + "l_tax", + "l_returnflag", + "l_linestatus", + "l_shipdate", + "l_commitdate", + "l_receiptdate", + "l_shipinstruct", + "l_shipmode", + "l_comment" + ], + "struct": { + "types": [ + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "i64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "fp64": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "date": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + }, + { + "string": { + "nullability": "NULLABILITY_NULLABLE" + } + } + ] + }, + "columnTypes": [ + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL", + "NORMAL_COL" + ] + }, + "advancedExtension": { + "optimization": { + "@type": "type.googleapis.com/google.protobuf.StringValue", + "value": "isMergeTree=0\n" + } + } + } + } + } + }, + "outputSchema": { + "nullability": "NULLABILITY_REQUIRED" + } + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json b/cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json new file mode 100644 index 000000000000..1471adf0a25f --- /dev/null +++ b/cpp-ch/local-engine/tests/json/mergetree/lineitem_parquet_input.json @@ -0,0 +1,28 @@ +{ + "items": [ + { + "uriFile": "{replace_GLUTEN_SOURCE_TPCH_DIR}lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet", + "length": "19230111", + "parquet": {}, + "schema": {}, + "metadataColumns": [ + { + "key": "input_file_name", + "value": "{replace_GLUTEN_SOURCE_TPCH_DIR}lineitem/part-00000-d08071cb-0dfa-42dc-9198-83cb334ccda3-c000.snappy.parquet" + }, + { + "key": "input_file_block_length", + "value": "19230111" + }, + { + "key": "input_file_block_start", + "value": "0" + } + ], + "properties": { + "fileSize": "19230111", + "modificationTime": "1719819166344" + } + } + ] +} \ No newline at end of file diff --git a/cpp-ch/local-engine/tests/json/native_write_one_partition.json b/cpp-ch/local-engine/tests/json/native_write_one_partition.json index 45b3f60e41fa..d09df9bd9b47 100644 --- a/cpp-ch/local-engine/tests/json/native_write_one_partition.json +++ b/cpp-ch/local-engine/tests/json/native_write_one_partition.json @@ -7,8 +7,11 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=parquet\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "parquet" + }, + "parquet": {} }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", diff --git a/cpp-ch/local-engine/tests/json/native_write_plan.json b/cpp-ch/local-engine/tests/json/native_write_plan.json index 8d5ffce7b163..c7142fb0e843 100644 --- a/cpp-ch/local-engine/tests/json/native_write_plan.json +++ b/cpp-ch/local-engine/tests/json/native_write_plan.json @@ -7,8 +7,11 @@ "namedTable": { "advancedExtension": { "optimization": { - "@type": "type.googleapis.com/google.protobuf.StringValue", - "value": "WriteParameters:isSnappy=1;format=parquet\n" + "@type": "type.googleapis.com/local_engine.Write", + "common": { + "format": "parquet" + }, + "parquet": {} }, "enhancement": { "@type": "type.googleapis.com/substrait.Type", diff --git a/cpp/CMake/Buildjemalloc_pic.cmake b/cpp/CMake/Buildjemalloc.cmake similarity index 79% rename from cpp/CMake/Buildjemalloc_pic.cmake rename to cpp/CMake/Buildjemalloc.cmake index 7c2316ea9540..67a0cc5f1167 100644 --- a/cpp/CMake/Buildjemalloc_pic.cmake +++ b/cpp/CMake/Buildjemalloc.cmake @@ -22,14 +22,14 @@ macro(build_jemalloc) if(DEFINED ENV{GLUTEN_JEMALLOC_URL}) set(JEMALLOC_SOURCE_URL "$ENV{GLUTEN_JEMALLOC_URL}") else() - set(JEMALLOC_BUILD_VERSION "5.2.1") + set(JEMALLOC_BUILD_VERSION "5.3.0") set(JEMALLOC_SOURCE_URL "https://github.com/jemalloc/jemalloc/releases/download/${JEMALLOC_BUILD_VERSION}/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" "https://github.com/ursa-labs/thirdparty/releases/download/latest/jemalloc-${JEMALLOC_BUILD_VERSION}.tar.bz2" ) endif() - set(JEMALLOC_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/jemalloc_ep-install") + set(JEMALLOC_PREFIX "/usr/local") set(JEMALLOC_LIB_DIR "${JEMALLOC_PREFIX}/lib") set(JEMALLOC_INCLUDE_DIR "${JEMALLOC_PREFIX}/include") set(JEMALLOC_STATIC_LIB @@ -41,12 +41,14 @@ macro(build_jemalloc) "CC=${CMAKE_C_COMPILER}" "--prefix=${JEMALLOC_PREFIX}" "--libdir=${JEMALLOC_LIB_DIR}" - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" + # The below prefix can be enabled if jemalloc is used in some selective + # code for debugging or profiling. "--with-jemalloc-prefix=je_gluten_" + # "--with-private-namespace=je_gluten_private_" This should be commented + # for dynamically linking. "--without-export" "--disable-cxx" "--disable-libdl" + # Enable heap profiling and leak detection functionality. + "--enable-prof" # For fixing an issue when loading native lib: cannot allocate memory in # static TLS block. "--disable-initial-exec-tls" @@ -64,11 +66,11 @@ macro(build_jemalloc) INSTALL_COMMAND make install) file(MAKE_DIRECTORY "${JEMALLOC_INCLUDE_DIR}") - add_library(jemalloc::libjemalloc STATIC IMPORTED) + add_library(jemalloc::jemalloc SHARED IMPORTED) set_target_properties( - jemalloc::libjemalloc + jemalloc::jemalloc PROPERTIES INTERFACE_LINK_LIBRARIES Threads::Threads - IMPORTED_LOCATION "${JEMALLOC_STATIC_LIB}" + IMPORTED_LOCATION "${JEMALLOC_LIB_DIR}/libjemalloc.so" INTERFACE_INCLUDE_DIRECTORIES "${JEMALLOC_INCLUDE_DIR}") - add_dependencies(jemalloc::libjemalloc jemalloc_ep) + add_dependencies(jemalloc::jemalloc jemalloc_ep) endmacro() diff --git a/cpp/CMake/Findjemalloc_pic.cmake b/cpp/CMake/Findjemalloc.cmake similarity index 76% rename from cpp/CMake/Findjemalloc_pic.cmake rename to cpp/CMake/Findjemalloc.cmake index ca7b7d213dfc..aa92fe26df29 100644 --- a/cpp/CMake/Findjemalloc_pic.cmake +++ b/cpp/CMake/Findjemalloc.cmake @@ -15,26 +15,21 @@ # specific language governing permissions and limitations # under the License. -# Find Jemalloc macro(find_jemalloc) - # Find the existing jemalloc - set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") - # Find from vcpkg-installed lib path. - find_library( - JEMALLOC_LIBRARY - NAMES jemalloc_pic - PATHS - ${CMAKE_CURRENT_BINARY_DIR}/../../../dev/vcpkg/vcpkg_installed/x64-linux-avx/lib/ - NO_DEFAULT_PATH) + set(SHARED_LIBRARY_SUFFIX ".so") + set(LIB_NAME "jemalloc") + set(LIB_FULL_NAME + ${CMAKE_SHARED_LIBRARY_PREFIX}${LIB_NAME}${SHARED_LIBRARY_SUFFIX}) + find_library(JEMALLOC_LIBRARY NAMES ${LIB_FULL_NAME}) if("${JEMALLOC_LIBRARY}" STREQUAL "JEMALLOC_LIBRARY-NOTFOUND") message(STATUS "Jemalloc Library Not Found.") set(JEMALLOC_NOT_FOUND TRUE) else() message(STATUS "Found jemalloc: ${JEMALLOC_LIBRARY}") find_path(JEMALLOC_INCLUDE_DIR jemalloc/jemalloc.h) - add_library(jemalloc::libjemalloc STATIC IMPORTED) + add_library(jemalloc::jemalloc SHARED IMPORTED) set_target_properties( - jemalloc::libjemalloc + jemalloc::jemalloc PROPERTIES INTERFACE_INCLUDE_DIRECTORIES "${JEMALLOC_INCLUDE_DIR}" IMPORTED_LOCATION "${JEMALLOC_LIBRARY}") endif() diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index ff934a2a8e14..5cc1d4ee4b19 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -48,7 +48,6 @@ option(BUILD_EXAMPLES "Build Examples" OFF) option(BUILD_BENCHMARKS "Build Benchmarks" OFF) option(ENABLE_JEMALLOC_STATS "Prints Jemalloc stats for debugging" OFF) option(BUILD_GLOG "Build Glog from Source" OFF) -option(USE_AVX512 "Build with AVX-512 optimizations" OFF) option(ENABLE_HBM "Enable HBM allocator" OFF) option(ENABLE_QAT "Enable QAT for de/compression" OFF) option(ENABLE_IAA "Enable IAA for de/compression" OFF) @@ -99,7 +98,9 @@ set(KNOWN_WARNINGS -Wno-error=unused-function \ -Wno-error=unused-variable \ -Wno-strict-aliasing \ - -Wno-ignored-qualifiers") + -Wno-ignored-qualifiers \ + -Wno-deprecated-declarations \ + -Wno-attributes") if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "GNU") set(KNOWN_WARNINGS "-Wno-error=unused-but-set-variable \ @@ -142,8 +143,25 @@ if(${CMAKE_SYSTEM_NAME} MATCHES "Darwin") add_compile_definitions(_GNU_SOURCE) endif() +if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") +endif() + set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${KNOWN_WARNINGS}") +# Keep same compile option with Velox. +execute_process( + COMMAND + bash -c + "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" + OUTPUT_VARIABLE SCRIPT_CXX_FLAGS + RESULT_VARIABLE COMMAND_STATUS) +if(COMMAND_STATUS EQUAL "1") + message(FATAL_ERROR "Unable to determine compiler flags!") +endif() + +set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") + # # Dependencies # @@ -218,14 +236,7 @@ if(ENABLE_IAA) add_definitions(-DGLUTEN_ENABLE_IAA) endif() -if(ENABLE_ORC) - add_definitions(-DGLUTEN_ENABLE_ORC) -endif() - -# # Subdirectories -# - add_subdirectory(core) if(BUILD_VELOX_BACKEND) diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index 14b9b2d6f249..5a5eeac35480 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -22,10 +22,7 @@ include(FindPkgConfig) include(GNUInstallDirs) include(CheckCXXCompilerFlag) -# Only set arch=native for non-AppleClang compilers. -if(NOT CMAKE_CXX_COMPILER_ID MATCHES "AppleClang") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -march=native") -endif() +message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") set(BOOST_MIN_VERSION "1.42.0") find_package(Boost REQUIRED) @@ -53,8 +50,10 @@ if(CCACHE_FOUND) endif(CCACHE_FOUND) macro(find_protobuf) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(Protobuf) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) if("${Protobuf_LIBRARY}" STREQUAL "Protobuf_LIBRARY-NOTFOUND") message(FATAL_ERROR "Protobuf Library Not Found") endif() @@ -64,20 +63,6 @@ macro(find_protobuf) CACHE PATH "Protobuf include path") endmacro() -if(USE_AVX512) - # Only enable additional instruction sets if they are supported - message(STATUS "System processor: ${CMAKE_SYSTEM_PROCESSOR}") - if(CMAKE_SYSTEM_PROCESSOR MATCHES "(x86)|(X86)|(amd64)|(AMD64)") - set(AVX512_FLAG "-march=skylake-avx512") - check_cxx_compiler_flag(${AVX512_FLAG} CXX_SUPPORTS_AVX512) - if(NOT CXX_SUPPORTS_AVX512) - message(FATAL_ERROR "AVX512 required but compiler doesn't support it.") - endif() - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${AVX512_FLAG}") - add_definitions(-DCOLUMNAR_PLUGIN_USE_AVX512) - endif() -endif() - # Set up Proto set(PROTO_OUTPUT_DIR "${CMAKE_CURRENT_BINARY_DIR}/proto") file(MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/proto) @@ -111,11 +96,6 @@ set_source_files_properties(${GLUTEN_PROTO_OUTPUT_FILES} PROPERTIES GENERATED TRUE) get_filename_component(GLUTEN_PROTO_DIR ${GLUTEN_PROTO_SRC_DIR}/ DIRECTORY) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") - -message("Core module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") - set(SPARK_COLUMNAR_PLUGIN_SRCS ${SUBSTRAIT_PROTO_SRCS} ${GLUTEN_PROTO_SRCS} diff --git a/cpp/core/compute/Runtime.h b/cpp/core/compute/Runtime.h index a58f770ff74c..16acada54fb3 100644 --- a/cpp/core/compute/Runtime.h +++ b/cpp/core/compute/Runtime.h @@ -27,6 +27,7 @@ #include "operators/c2r/ColumnarToRow.h" #include "operators/r2c/RowToColumnar.h" #include "operators/serializer/ColumnarBatchSerializer.h" +#include "operators/writer/ArrowWriter.h" #include "shuffle/ShuffleReader.h" #include "shuffle/ShuffleWriter.h" #include "substrait/plan.pb.h" @@ -79,11 +80,17 @@ class Runtime : public std::enable_shared_from_this { return kind_; } - virtual void parsePlan(const uint8_t* data, int32_t size, std::optional dumpFile) = 0; + virtual void parsePlan(const uint8_t* data, int32_t size, std::optional dumpFile) { + throw GlutenException("Not implemented"); + } - virtual void parseSplitInfo(const uint8_t* data, int32_t size, std::optional dumpFile) = 0; + virtual void parseSplitInfo(const uint8_t* data, int32_t size, std::optional dumpFile) { + throw GlutenException("Not implemented"); + } - virtual std::string planString(bool details, const std::unordered_map& sessionConf) = 0; + virtual std::string planString(bool details, const std::unordered_map& sessionConf) { + throw GlutenException("Not implemented"); + } // Just for benchmark ::substrait::Plan& getPlan() { @@ -93,11 +100,17 @@ class Runtime : public std::enable_shared_from_this { virtual std::shared_ptr createResultIterator( const std::string& spillDir, const std::vector>& inputs, - const std::unordered_map& sessionConf) = 0; + const std::unordered_map& sessionConf) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) = 0; + virtual std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr select(std::shared_ptr, const std::vector&) = 0; + virtual std::shared_ptr select(std::shared_ptr, const std::vector&) { + throw GlutenException("Not implemented"); + } virtual MemoryManager* memoryManager() { return memoryManager_; @@ -105,24 +118,42 @@ class Runtime : public std::enable_shared_from_this { /// This function is used to create certain converter from the format used by /// the backend to Spark unsafe row. - virtual std::shared_ptr createColumnar2RowConverter(int64_t column2RowMemThreshold) = 0; + virtual std::shared_ptr createColumnar2RowConverter(int64_t column2RowMemThreshold) { + throw GlutenException("Not implemented"); + } - virtual std::shared_ptr createRow2ColumnarConverter(struct ArrowSchema* cSchema) = 0; + virtual std::shared_ptr createRow2ColumnarConverter(struct ArrowSchema* cSchema) { + throw GlutenException("Not implemented"); + } virtual std::shared_ptr createShuffleWriter( int numPartitions, std::unique_ptr partitionWriter, - ShuffleWriterOptions options) = 0; + ShuffleWriterOptions options) { + throw GlutenException("Not implemented"); + } - virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) = 0; + virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) { + throw GlutenException("Not implemented"); + } virtual std::shared_ptr createShuffleReader( std::shared_ptr schema, - ShuffleReaderOptions options) = 0; + ShuffleReaderOptions options) { + throw GlutenException("Not implemented"); + } - virtual std::unique_ptr createColumnarBatchSerializer(struct ArrowSchema* cSchema) = 0; + virtual std::unique_ptr createColumnarBatchSerializer(struct ArrowSchema* cSchema) { + throw GlutenException("Not implemented"); + } - virtual void dumpConf(const std::string& path) = 0; + virtual void dumpConf(const std::string& path) { + throw GlutenException("Not implemented"); + } + + virtual std::shared_ptr createArrowWriter(const std::string& path) { + throw GlutenException("Not implemented"); + }; const std::unordered_map& getConfMap() { return confMap_; diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h index 47e5558f280e..5a61b27a80b6 100644 --- a/cpp/core/config/GlutenConfig.h +++ b/cpp/core/config/GlutenConfig.h @@ -18,6 +18,7 @@ #pragma once #include +#include #include #include @@ -72,6 +73,7 @@ const std::string kSparkRedactionRegex = "spark.redaction.regex"; const std::string kSparkRedactionString = "*********(redacted)"; const std::string kSparkLegacyTimeParserPolicy = "spark.sql.legacy.timeParserPolicy"; +const std::string kShuffleFileBufferSize = "spark.shuffle.file.buffer"; std::unordered_map parseConfMap(JNIEnv* env, const uint8_t* planData, const int32_t planDataLength); diff --git a/cpp/core/jni/JniCommon.cc b/cpp/core/jni/JniCommon.cc index 46b53e8c3b00..bb8554568ba0 100644 --- a/cpp/core/jni/JniCommon.cc +++ b/cpp/core/jni/JniCommon.cc @@ -104,22 +104,34 @@ gluten::JniColumnarBatchIterator::~JniColumnarBatchIterator() { std::shared_ptr gluten::JniColumnarBatchIterator::next() { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm_, &env); - if (!env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { - checkException(env); - return nullptr; // stream ended - } - - checkException(env); - jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); - checkException(env); - auto batch = ObjectStore::retrieve(handle); if (writer_ != nullptr) { - // save snapshot of the batch to file - std::shared_ptr schema = batch->exportArrowSchema(); - std::shared_ptr array = batch->exportArrowArray(); - auto rb = gluten::arrowGetOrThrow(arrow::ImportRecordBatch(array.get(), schema.get())); - GLUTEN_THROW_NOT_OK(writer_->initWriter(*(rb->schema().get()))); - GLUTEN_THROW_NOT_OK(writer_->writeInBatches(rb)); + if (!writer_->closed()) { + // Dump all inputs. + while (env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { + checkException(env); + jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); + checkException(env); + auto batch = ObjectStore::retrieve(handle); + + // Save the snapshot of the batch to file. + std::shared_ptr schema = batch->exportArrowSchema(); + std::shared_ptr array = batch->exportArrowArray(); + auto rb = gluten::arrowGetOrThrow(arrow::ImportRecordBatch(array.get(), schema.get())); + GLUTEN_THROW_NOT_OK(writer_->initWriter(*(rb->schema().get()))); + GLUTEN_THROW_NOT_OK(writer_->writeInBatches(rb)); + } + checkException(env); + GLUTEN_THROW_NOT_OK(writer_->closeWriter()); + } + return writer_->retrieveColumnarBatch(); + } else { + if (!env->CallBooleanMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorHasNext_)) { + checkException(env); + return nullptr; // stream ended + } + checkException(env); + jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext_); + checkException(env); + return ObjectStore::retrieve(handle); } - return batch; } diff --git a/cpp/core/jni/JniCommon.h b/cpp/core/jni/JniCommon.h index aeab454f1aab..8f40398a4132 100644 --- a/cpp/core/jni/JniCommon.h +++ b/cpp/core/jni/JniCommon.h @@ -26,6 +26,7 @@ #include "compute/Runtime.h" #include "config/GlutenConfig.h" #include "memory/AllocationListener.h" +#include "operators/writer/ArrowWriter.h" #include "shuffle/rss/RssClient.h" #include "utils/Compression.h" #include "utils/Exception.h" diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index 45f19c25c749..f75b16b46126 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -41,32 +41,33 @@ using namespace gluten; -static jclass javaReservationListenerClass; +namespace { +jclass javaReservationListenerClass; -static jmethodID reserveMemoryMethod; -static jmethodID unreserveMemoryMethod; +jmethodID reserveMemoryMethod; +jmethodID unreserveMemoryMethod; -static jclass byteArrayClass; +jclass byteArrayClass; -static jclass jniByteInputStreamClass; -static jmethodID jniByteInputStreamRead; -static jmethodID jniByteInputStreamTell; -static jmethodID jniByteInputStreamClose; +jclass jniByteInputStreamClass; +jmethodID jniByteInputStreamRead; +jmethodID jniByteInputStreamTell; +jmethodID jniByteInputStreamClose; -static jclass splitResultClass; -static jmethodID splitResultConstructor; +jclass splitResultClass; +jmethodID splitResultConstructor; -static jclass columnarBatchSerializeResultClass; -static jmethodID columnarBatchSerializeResultConstructor; +jclass columnarBatchSerializeResultClass; +jmethodID columnarBatchSerializeResultConstructor; -static jclass metricsBuilderClass; -static jmethodID metricsBuilderConstructor; -static jclass nativeColumnarToRowInfoClass; -static jmethodID nativeColumnarToRowInfoConstructor; +jclass metricsBuilderClass; +jmethodID metricsBuilderConstructor; +jclass nativeColumnarToRowInfoClass; +jmethodID nativeColumnarToRowInfoConstructor; -static jclass shuffleReaderMetricsClass; -static jmethodID shuffleReaderMetricsSetDecompressTime; -static jmethodID shuffleReaderMetricsSetDeserializeTime; +jclass shuffleReaderMetricsClass; +jmethodID shuffleReaderMetricsSetDecompressTime; +jmethodID shuffleReaderMetricsSetDeserializeTime; class JavaInputStreamAdaptor final : public arrow::io::InputStream { public: @@ -140,6 +141,61 @@ class JavaInputStreamAdaptor final : public arrow::io::InputStream { bool closed_ = false; }; +/// Internal backend consists of empty implementations of Runtime API and MemoryManager API. +/// The backend is used for saving contextual objects only. +/// +/// It's also possible to extend the implementation for handling Arrow-based requests either in the future. +inline static const std::string kInternalBackendKind{"internal"}; + +class InternalMemoryManager : public MemoryManager { + public: + InternalMemoryManager(const std::string& kind) : MemoryManager(kind) {} + + arrow::MemoryPool* getArrowMemoryPool() override { + throw GlutenException("Not implemented"); + } + + const MemoryUsageStats collectMemoryUsageStats() const override { + return MemoryUsageStats(); + } + + const int64_t shrink(int64_t size) override { + return 0; + } + + void hold() override {} +}; + +class InternalRuntime : public Runtime { + public: + InternalRuntime( + const std::string& kind, + MemoryManager* memoryManager, + const std::unordered_map& confMap) + : Runtime(kind, memoryManager, confMap) {} +}; + +MemoryManager* internalMemoryManagerFactory(const std::string& kind, std::unique_ptr listener) { + return new InternalMemoryManager(kind); +} + +void internalMemoryManagerReleaser(MemoryManager* memoryManager) { + delete memoryManager; +} + +Runtime* internalRuntimeFactory( + const std::string& kind, + MemoryManager* memoryManager, + const std::unordered_map& sessionConf) { + return new InternalRuntime(kind, memoryManager, sessionConf); +} + +void internalRuntimeReleaser(Runtime* runtime) { + delete runtime; +} + +} // namespace + #ifdef __cplusplus extern "C" { #endif @@ -152,6 +208,9 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { getJniCommonState()->ensureInitialized(env); getJniErrorState()->ensureInitialized(env); + MemoryManager::registerFactory(kInternalBackendKind, internalMemoryManagerFactory, internalMemoryManagerReleaser); + Runtime::registerFactory(kInternalBackendKind, internalRuntimeFactory, internalRuntimeReleaser); + byteArrayClass = createGlobalClassReferenceOrError(env, "[B"); jniByteInputStreamClass = createGlobalClassReferenceOrError(env, "Lorg/apache/gluten/vectorized/JniByteInputStream;"); @@ -275,11 +334,11 @@ JNIEXPORT jbyteArray JNICALL Java_org_apache_gluten_memory_NativeMemoryManagerJn const MemoryUsageStats& stats = memoryManager->collectMemoryUsageStats(); auto size = stats.ByteSizeLong(); jbyteArray out = env->NewByteArray(size); - uint8_t buffer[size]; + std::vector buffer(size); GLUTEN_CHECK( - stats.SerializeToArray(reinterpret_cast(buffer), size), + stats.SerializeToArray(reinterpret_cast(buffer.data()), size), "Serialization failed when collecting memory usage stats"); - env->SetByteArrayRegion(out, 0, size, reinterpret_cast(buffer)); + env->SetByteArrayRegion(out, 0, size, reinterpret_cast(buffer.data())); return out; JNI_METHOD_END(nullptr) } @@ -373,8 +432,16 @@ Java_org_apache_gluten_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWith } saveDir = conf.at(kGlutenSaveDir); std::filesystem::path f{saveDir}; - if (!std::filesystem::exists(f)) { - throw GlutenException("Save input path " + saveDir + " does not exists"); + if (std::filesystem::exists(f)) { + if (!std::filesystem::is_directory(f)) { + throw GlutenException("Invalid path for " + kGlutenSaveDir + ": " + saveDir); + } + } else { + std::error_code ec; + std::filesystem::create_directory(f, ec); + if (ec) { + throw GlutenException("Failed to create directory: " + saveDir + ", error message: " + ec.message()); + } } ctx->dumpConf(saveDir + "/conf" + fileIdentifier + ".ini"); } @@ -407,7 +474,7 @@ Java_org_apache_gluten_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWith std::shared_ptr writer = nullptr; if (saveInput) { auto file = saveDir + "/data" + fileIdentifier + "_" + std::to_string(idx) + ".parquet"; - writer = std::make_shared(file); + writer = ctx->createArrowWriter(file); } jobject iter = env->GetObjectArrayElement(iterArr, idx); auto arrayIter = makeJniColumnarBatchIterator(env, iter, ctx, writer); @@ -467,7 +534,7 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_metrics_IteratorMetricsJniWrapp } jlongArray longArray[Metrics::kNum]; - for (auto i = (int)Metrics::kBegin; i != (int)Metrics::kEnd; ++i) { + for (auto i = static_cast(Metrics::kBegin); i != static_cast(Metrics::kEnd); ++i) { longArray[i] = env->NewLongArray(numMetrics); if (metrics) { env->SetLongArrayRegion(longArray[i], 0, numMetrics, metrics->get((Metrics::TYPE)i)); @@ -642,7 +709,7 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_vectorized_NativeRowToColumnarJniW JNIEXPORT jstring JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_getType( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -652,7 +719,7 @@ JNIEXPORT jstring JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniW JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numBytes( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -662,7 +729,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numColumns( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -672,7 +739,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_numRows( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle) { JNI_METHOD_START auto batch = ObjectStore::retrieve(batchHandle); @@ -682,7 +749,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_exportToArrow( // NOLINT JNIEnv* env, - jobject wrapper, + jclass, jlong batchHandle, jlong cSchema, jlong cArray) { @@ -695,6 +762,15 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrap JNI_METHOD_END() } +JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_close( // NOLINT + JNIEnv* env, + jclass, + jlong batchHandle) { + JNI_METHOD_START + ObjectStore::release(batchHandle); + JNI_METHOD_END() +} + JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_createWithArrowArray( // NOLINT JNIEnv* env, jobject wrapper, @@ -744,15 +820,6 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWra JNI_METHOD_END(kInvalidObjectHandle) } -JNIEXPORT void JNICALL Java_org_apache_gluten_columnarbatch_ColumnarBatchJniWrapper_close( // NOLINT - JNIEnv* env, - jobject wrapper, - jlong batchHandle) { - JNI_METHOD_START - ObjectStore::release(batchHandle); - JNI_METHOD_END() -} - // Shuffle JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrapper_nativeMake( // NOLINT JNIEnv* env, @@ -793,7 +860,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe .bufferSize = bufferSize, .bufferReallocThreshold = reallocThreshold, .partitioning = toPartitioning(jStringToCString(env, partitioningNameJstr)), - .taskAttemptId = (int64_t)taskAttemptId, + .taskAttemptId = static_cast(taskAttemptId), .startPartitionId = startPartitionId, .shuffleWriterType = ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterTypeJstr)), .sortBufferInitialSize = sortBufferInitialSize, @@ -816,6 +883,13 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleWriterJniWrappe partitionWriterOptions.codecBackend = getCodecBackend(env, codecBackendJstr); partitionWriterOptions.compressionMode = getCompressionMode(env, compressionModeJstr); } + const auto& conf = ctx->getConfMap(); + { + auto it = conf.find(kShuffleFileBufferSize); + if (it != conf.end()) { + partitionWriterOptions.shuffleFileBufferSize = static_cast(stoi(it->second)); + } + } std::unique_ptr partitionWriter; @@ -996,6 +1070,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe jstring compressionType, jstring compressionBackend, jint batchSize, + jlong bufferSize, jstring shuffleWriterType) { JNI_METHOD_START auto ctx = getRuntime(env, wrapper); @@ -1007,7 +1082,7 @@ JNIEXPORT jlong JNICALL Java_org_apache_gluten_vectorized_ShuffleReaderJniWrappe options.codecBackend = getCodecBackend(env, compressionBackend); } options.batchSize = batchSize; - // TODO: Add coalesce option and maximum coalesced size. + options.bufferSize = bufferSize; options.shuffleWriterType = ShuffleWriter::stringToType(jStringToCString(env, shuffleWriterType)); std::shared_ptr schema = @@ -1079,6 +1154,10 @@ JNIEXPORT jobject JNICALL Java_org_apache_gluten_vectorized_ColumnarBatchSeriali auto serializer = ctx->createColumnarBatchSerializer(nullptr); auto buffer = serializer->serializeColumnarBatches(batches); auto bufferArr = env->NewByteArray(buffer->size()); + GLUTEN_CHECK( + bufferArr != nullptr, + "Cannot construct a byte array of size " + std::to_string(buffer->size()) + + " byte(s) to serialize columnar batches"); env->SetByteArrayRegion(bufferArr, 0, buffer->size(), reinterpret_cast(buffer->data())); jobject columnarBatchSerializeResult = diff --git a/cpp/core/memory/ColumnarBatch.h b/cpp/core/memory/ColumnarBatch.h index e0bab254189a..be487f871e74 100644 --- a/cpp/core/memory/ColumnarBatch.h +++ b/cpp/core/memory/ColumnarBatch.h @@ -23,7 +23,6 @@ #include "arrow/c/helpers.h" #include "arrow/record_batch.h" #include "memory/MemoryManager.h" -#include "operators/writer/ArrowWriter.h" #include "utils/ArrowStatus.h" #include "utils/Exception.h" diff --git a/cpp/core/operators/writer/ArrowWriter.cc b/cpp/core/operators/writer/ArrowWriter.cc index 19bab6ddcba3..46ec2fc9ba6c 100644 --- a/cpp/core/operators/writer/ArrowWriter.cc +++ b/cpp/core/operators/writer/ArrowWriter.cc @@ -21,6 +21,7 @@ #include "arrow/table.h" #include "arrow/util/type_fwd.h" +namespace gluten { arrow::Status ArrowWriter::initWriter(arrow::Schema& schema) { if (writer_ != nullptr) { return arrow::Status::OK(); @@ -50,9 +51,15 @@ arrow::Status ArrowWriter::writeInBatches(std::shared_ptr ba } arrow::Status ArrowWriter::closeWriter() { - // Write file footer and close + // Write file footer and close. if (writer_ != nullptr) { ARROW_RETURN_NOT_OK(writer_->Close()); } + closed_ = true; return arrow::Status::OK(); } + +bool ArrowWriter::closed() const { + return closed_; +} +} // namespace gluten diff --git a/cpp/core/operators/writer/ArrowWriter.h b/cpp/core/operators/writer/ArrowWriter.h index 0d0b8ce2cb58..1a7b19606624 100644 --- a/cpp/core/operators/writer/ArrowWriter.h +++ b/cpp/core/operators/writer/ArrowWriter.h @@ -17,15 +17,19 @@ #pragma once -#include "parquet/arrow/writer.h" +#include +#include "memory/ColumnarBatch.h" +namespace gluten { /** * @brief Used to print RecordBatch to a parquet file * */ class ArrowWriter { public: - explicit ArrowWriter(std::string& path) : path_(path) {} + explicit ArrowWriter(const std::string& path) : path_(path) {} + + virtual ~ArrowWriter() = default; arrow::Status initWriter(arrow::Schema& schema); @@ -33,7 +37,13 @@ class ArrowWriter { arrow::Status closeWriter(); - private: + bool closed() const; + + virtual std::shared_ptr retrieveColumnarBatch() = 0; + + protected: std::unique_ptr writer_; std::string path_; + bool closed_{false}; }; +} // namespace gluten diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index f0edfa257357..6692097986c2 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -379,7 +379,7 @@ LocalPartitionWriter::LocalPartitionWriter( } std::string LocalPartitionWriter::nextSpilledFileDir() { - auto spilledFileDir = getSpilledShuffleFileDir(localDirs_[dirSelection_], subDirSelection_[dirSelection_]); + auto spilledFileDir = getShuffleSpillDir(localDirs_[dirSelection_], subDirSelection_[dirSelection_]); subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % options_.numSubDirs; dirSelection_ = (dirSelection_ + 1) % localDirs_.size(); return spilledFileDir; @@ -389,9 +389,9 @@ arrow::Result> LocalPartitionWriter::op std::shared_ptr fout; ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(file)); if (options_.bufferedWrite) { - // The 16k bytes is a temporary allocation and will be freed with file close. + // The `shuffleFileBufferSize` bytes is a temporary allocation and will be freed with file close. // Use default memory pool and count treat the memory as executor memory overhead to avoid unnecessary spill. - return arrow::io::BufferedOutputStream::Create(16384, arrow::default_memory_pool(), fout); + return arrow::io::BufferedOutputStream::Create(options_.shuffleFileBufferSize, arrow::default_memory_pool(), fout); } return fout; } @@ -420,6 +420,7 @@ arrow::Status LocalPartitionWriter::mergeSpills(uint32_t partitionId) { auto spillIter = spills_.begin(); while (spillIter != spills_.end()) { ARROW_ASSIGN_OR_RAISE(auto st, dataFileOs_->Tell()); + (*spillIter)->openForRead(options_.shuffleFileBufferSize); // Read if partition exists in the spilled file and write to the final file. while (auto payload = (*spillIter)->nextPayload(partitionId)) { // May trigger spill during compression. @@ -505,6 +506,9 @@ arrow::Status LocalPartitionWriter::stop(ShuffleWriterMetrics* metrics) { "Merging from spill " + std::to_string(s) + " is not exhausted. pid: " + std::to_string(pid)); } } + if (std::filesystem::exists(spill->spillFile()) && !std::filesystem::remove(spill->spillFile())) { + LOG(WARNING) << "Error while deleting spill file " << spill->spillFile(); + } ++s; } spills_.clear(); diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 2b86a81e883b..3ed0a1306008 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -17,7 +17,6 @@ #pragma once -#include #include #include "shuffle/PartitionWriter.h" diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h index a3dc9f6260b0..3a1efdc2ae90 100644 --- a/cpp/core/shuffle/Options.h +++ b/cpp/core/shuffle/Options.h @@ -38,6 +38,8 @@ static constexpr double kDefaultMergeBufferThreshold = 0.25; static constexpr bool kEnableBufferedWrite = true; static constexpr bool kDefaultUseRadixSort = true; static constexpr int32_t kDefaultSortBufferSize = 4096; +static constexpr int64_t kDefaultReadBufferSize = 1 << 20; +static constexpr int64_t kDefaultShuffleFileBufferSize = 32 << 10; enum ShuffleWriterType { kHashShuffle, kSortShuffle, kRssSortShuffle }; enum PartitionWriterType { kLocal, kRss }; @@ -49,6 +51,7 @@ struct ShuffleReaderOptions { ShuffleWriterType shuffleWriterType = kHashShuffle; CodecBackend codecBackend = CodecBackend::NONE; int32_t batchSize = kDefaultBatchSize; + int64_t bufferSize = kDefaultReadBufferSize; }; struct ShuffleWriterOptions { @@ -84,6 +87,8 @@ struct PartitionWriterOptions { int64_t pushBufferMaxSize = kDefaultPushMemoryThreshold; int64_t sortBufferMaxSize = kDefaultSortBufferThreshold; + + int64_t shuffleFileBufferSize = kDefaultShuffleFileBufferSize; }; struct ShuffleWriterMetrics { diff --git a/cpp/core/shuffle/Payload.cc b/cpp/core/shuffle/Payload.cc index d5cdb6d08eda..ddf4a409661e 100644 --- a/cpp/core/shuffle/Payload.cc +++ b/cpp/core/shuffle/Payload.cc @@ -118,9 +118,9 @@ arrow::Status compressAndFlush( return arrow::Status::OK(); } -arrow::Result> readUncompressedBuffer( - arrow::io::InputStream* inputStream, - arrow::MemoryPool* pool) { +arrow::Result> +readUncompressedBuffer(arrow::io::InputStream* inputStream, arrow::MemoryPool* pool, int64_t& deserializedTime) { + ScopedTimer timer(&deserializedTime); int64_t bufferLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &bufferLength)); if (bufferLength == kNullBuffer) { @@ -135,7 +135,9 @@ arrow::Result> readCompressedBuffer( arrow::io::InputStream* inputStream, const std::shared_ptr& codec, arrow::MemoryPool* pool, + int64_t& deserializeTime, int64_t& decompressTime) { + ScopedTimer timer(&deserializeTime); int64_t compressedLength; RETURN_NOT_OK(inputStream->Read(sizeof(int64_t), &compressedLength)); if (compressedLength == kNullBuffer) { @@ -155,7 +157,7 @@ arrow::Result> readCompressedBuffer( ARROW_ASSIGN_OR_RAISE(auto compressed, arrow::AllocateResizableBuffer(compressedLength, pool)); RETURN_NOT_OK(inputStream->Read(compressedLength, compressed->mutable_data())); - ScopedTimer timer(&decompressTime); + timer.switchTo(&decompressTime); ARROW_ASSIGN_OR_RAISE(auto output, arrow::AllocateResizableBuffer(uncompressedLength, pool)); RETURN_NOT_OK(codec->Decompress(compressedLength, compressed->data(), uncompressedLength, output->mutable_data())); return output; @@ -296,7 +298,9 @@ arrow::Result>> BlockPayload::deseria const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime) { + auto timer = std::make_unique(&deserializeTime); static const std::vector> kEmptyBuffers{}; ARROW_ASSIGN_OR_RAISE(auto type, readType(inputStream)); if (type == 0) { @@ -306,6 +310,7 @@ arrow::Result>> BlockPayload::deseria RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numRows)); uint32_t numBuffers; RETURN_NOT_OK(inputStream->Read(sizeof(uint32_t), &numBuffers)); + timer.reset(); bool isCompressionEnabled = type == Type::kCompressed; std::vector> buffers; @@ -313,9 +318,10 @@ arrow::Result>> BlockPayload::deseria for (auto i = 0; i < numBuffers; ++i) { buffers.emplace_back(); if (isCompressionEnabled) { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readCompressedBuffer(inputStream, codec, pool, decompressTime)); + ARROW_ASSIGN_OR_RAISE( + buffers.back(), readCompressedBuffer(inputStream, codec, pool, deserializeTime, decompressTime)); } else { - ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool)); + ARROW_ASSIGN_OR_RAISE(buffers.back(), readUncompressedBuffer(inputStream, pool, deserializeTime)); } } return buffers; @@ -475,6 +481,9 @@ arrow::Result> UncompressedDiskBlockPayload::read } arrow::Status UncompressedDiskBlockPayload::serialize(arrow::io::OutputStream* outputStream) { + ARROW_RETURN_IF( + inputStream_ == nullptr, arrow::Status::Invalid("inputStream_ is uninitialized before calling serialize().")); + if (codec_ == nullptr || type_ == Payload::kUncompressed) { ARROW_ASSIGN_OR_RAISE(auto block, inputStream_->Read(rawSize_)); RETURN_NOT_OK(outputStream->Write(block)); @@ -539,6 +548,8 @@ CompressedDiskBlockPayload::CompressedDiskBlockPayload( : Payload(Type::kCompressed, numRows, isValidityBuffer), inputStream_(inputStream), rawSize_(rawSize) {} arrow::Status CompressedDiskBlockPayload::serialize(arrow::io::OutputStream* outputStream) { + ARROW_RETURN_IF( + inputStream_ == nullptr, arrow::Status::Invalid("inputStream_ is uninitialized before calling serialize().")); ScopedTimer timer(&writeTime_); ARROW_ASSIGN_OR_RAISE(auto block, inputStream_->Read(rawSize_)); RETURN_NOT_OK(outputStream->Write(block)); diff --git a/cpp/core/shuffle/Payload.h b/cpp/core/shuffle/Payload.h index ea8c897e96d0..611b2310d547 100644 --- a/cpp/core/shuffle/Payload.h +++ b/cpp/core/shuffle/Payload.h @@ -92,6 +92,7 @@ class BlockPayload final : public Payload { const std::shared_ptr& codec, arrow::MemoryPool* pool, uint32_t& numRows, + int64_t& deserializeTime, int64_t& decompressTime); static int64_t maxCompressedLength( diff --git a/cpp/core/shuffle/Spill.cc b/cpp/core/shuffle/Spill.cc index d8b9bc7ebf99..8cc3a9d05ea7 100644 --- a/cpp/core/shuffle/Spill.cc +++ b/cpp/core/shuffle/Spill.cc @@ -34,7 +34,6 @@ bool Spill::hasNextPayload(uint32_t partitionId) { } std::unique_ptr Spill::nextPayload(uint32_t partitionId) { - openSpillFile(); if (!hasNextPayload(partitionId)) { return nullptr; } @@ -71,9 +70,9 @@ void Spill::insertPayload( } } -void Spill::openSpillFile() { +void Spill::openForRead(uint64_t shuffleFileBufferSize) { if (!is_) { - GLUTEN_ASSIGN_OR_THROW(is_, arrow::io::MemoryMappedFile::Open(spillFile_, arrow::io::FileMode::READ)); + GLUTEN_ASSIGN_OR_THROW(is_, MmapFileStream::open(spillFile_, shuffleFileBufferSize)); rawIs_ = is_.get(); } } diff --git a/cpp/core/shuffle/Spill.h b/cpp/core/shuffle/Spill.h index c82a60f562b4..fd692537c570 100644 --- a/cpp/core/shuffle/Spill.h +++ b/cpp/core/shuffle/Spill.h @@ -37,6 +37,8 @@ class Spill final { SpillType type() const; + void openForRead(uint64_t shuffleFileBufferSize); + bool hasNextPayload(uint32_t partitionId); std::unique_ptr nextPayload(uint32_t partitionId); @@ -69,15 +71,12 @@ class Spill final { }; SpillType type_; - std::shared_ptr is_; + std::shared_ptr is_; std::list partitionPayloads_{}; - std::shared_ptr inputStream_{}; std::string spillFile_; int64_t spillTime_; int64_t compressTime_; - arrow::io::InputStream* rawIs_; - - void openSpillFile(); + arrow::io::InputStream* rawIs_{nullptr}; }; } // namespace gluten \ No newline at end of file diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index 6854c1978370..457702c0c9df 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -16,16 +16,19 @@ */ #include "shuffle/Utils.h" +#include #include -#include -#include #include +#include +#include +#include #include #include #include #include #include #include "shuffle/Options.h" +#include "utils/StringUtil.h" #include "utils/Timer.h" namespace gluten { @@ -151,6 +154,14 @@ arrow::Status getLengthBufferAndValueBufferStream( *compressedLengthPtr = actualLength; return arrow::Status::OK(); } + +uint64_t roundUpToPageSize(uint64_t value) { + static auto pageSize = static_cast(arrow::internal::GetPageSize()); + static auto pageMask = ~(pageSize - 1); + DCHECK_GT(pageSize, 0); + DCHECK_EQ(pageMask & pageSize, pageSize); + return (value + pageSize - 1) & pageMask; +} } // namespace arrow::Result> makeCompressedRecordBatch( @@ -212,19 +223,113 @@ arrow::Result> makeUncompressedRecordBatch( } return arrow::RecordBatch::Make(writeSchema, 1, {arrays}); } -} // namespace gluten -std::string gluten::generateUuid() { - boost::uuids::random_generator generator; - return boost::uuids::to_string(generator()); +MmapFileStream::MmapFileStream(arrow::internal::FileDescriptor fd, uint8_t* data, int64_t size, uint64_t prefetchSize) + : prefetchSize_(roundUpToPageSize(prefetchSize)), fd_(std::move(fd)), data_(data), size_(size){}; + +arrow::Result> MmapFileStream::open(const std::string& path, uint64_t prefetchSize) { + ARROW_ASSIGN_OR_RAISE(auto fileName, arrow::internal::PlatformFilename::FromString(path)); + + ARROW_ASSIGN_OR_RAISE(auto fd, arrow::internal::FileOpenReadable(fileName)); + ARROW_ASSIGN_OR_RAISE(auto size, arrow::internal::FileGetSize(fd.fd())); + + ARROW_RETURN_IF(size == 0, arrow::Status::Invalid("Cannot mmap an empty file: ", path)); + + void* result = mmap(nullptr, size, PROT_READ, MAP_PRIVATE, fd.fd(), 0); + if (result == MAP_FAILED) { + return arrow::Status::IOError("Memory mapping file failed: ", ::arrow::internal::ErrnoMessage(errno)); + } + + return std::make_shared(std::move(fd), static_cast(result), size, prefetchSize); +} + +arrow::Result MmapFileStream::actualReadSize(int64_t nbytes) { + if (nbytes < 0 || pos_ > size_) { + return arrow::Status::IOError("Read out of range. Offset: ", pos_, " Size: ", nbytes, " File Size: ", size_); + } + return std::min(size_ - pos_, nbytes); +} + +bool MmapFileStream::closed() const { + return data_ == nullptr; +}; + +void MmapFileStream::advance(int64_t length) { + // Dont need data before pos + auto purgeLength = (pos_ - posRetain_) / prefetchSize_ * prefetchSize_; + if (purgeLength > 0) { + int ret = madvise(data_ + posRetain_, purgeLength, MADV_DONTNEED); + if (ret != 0) { + LOG(WARNING) << "fadvise failed " << ::arrow::internal::ErrnoMessage(errno); + } + posRetain_ += purgeLength; + } + + pos_ += length; +} + +void MmapFileStream::willNeed(int64_t length) { + // Skip if already fetched + if (pos_ + length <= posFetch_) { + return; + } + + // Round up to multiple of prefetchSize + auto fetchLen = ((length + prefetchSize_ - 1) / prefetchSize_) * prefetchSize_; + fetchLen = std::min(size_ - pos_, fetchLen); + int ret = madvise(data_ + posFetch_, fetchLen, MADV_WILLNEED); + if (ret != 0) { + LOG(WARNING) << "madvise willneed failed: " << ::arrow::internal::ErrnoMessage(errno); + } + + posFetch_ += fetchLen; +} + +arrow::Status MmapFileStream::Close() { + if (data_ != nullptr) { + int result = munmap(data_, size_); + if (result != 0) { + LOG(WARNING) << "munmap failed"; + } + data_ = nullptr; + } + + return fd_.Close(); +} + +arrow::Result MmapFileStream::Tell() const { + return pos_; +} + +arrow::Result MmapFileStream::Read(int64_t nbytes, void* out) { + ARROW_ASSIGN_OR_RAISE(nbytes, actualReadSize(nbytes)); + + if (nbytes > 0) { + memcpy(out, data_ + pos_, nbytes); + advance(nbytes); + } + + return nbytes; } -std::string gluten::getSpilledShuffleFileDir(const std::string& configuredDir, int32_t subDirId) { - auto fs = std::make_shared(); +arrow::Result> MmapFileStream::Read(int64_t nbytes) { + ARROW_ASSIGN_OR_RAISE(nbytes, actualReadSize(nbytes)); + + if (nbytes > 0) { + auto buffer = std::make_shared(data_ + pos_, nbytes); + willNeed(nbytes); + advance(nbytes); + return buffer; + } else { + return std::make_shared(nullptr, 0); + } +} +} // namespace gluten + +std::string gluten::getShuffleSpillDir(const std::string& configuredDir, int32_t subDirId) { std::stringstream ss; ss << std::setfill('0') << std::setw(2) << std::hex << subDirId; - auto dir = arrow::fs::internal::ConcatAbstractPath(configuredDir, ss.str()); - return dir; + return std::filesystem::path(configuredDir) / ss.str(); } arrow::Result gluten::createTempShuffleFile(const std::string& dir) { @@ -232,22 +337,25 @@ arrow::Result gluten::createTempShuffleFile(const std::string& dir) return arrow::Status::Invalid("Failed to create spilled file, got empty path."); } - auto fs = std::make_shared(); - ARROW_ASSIGN_OR_RAISE(auto path_info, fs->GetFileInfo(dir)); - if (path_info.type() == arrow::fs::FileType::NotFound) { - RETURN_NOT_OK(fs->CreateDir(dir, true)); + if (std::filesystem::exists(dir)) { + if (!std::filesystem::is_directory(dir)) { + return arrow::Status::Invalid("Invalid directory. File path exists but is not a directory: ", dir); + } + } else { + std::filesystem::create_directories(dir); } + const auto parentPath = std::filesystem::path(dir); bool exist = true; - std::string filePath; + std::filesystem::path filePath; while (exist) { - filePath = arrow::fs::internal::ConcatAbstractPath(dir, "temp_shuffle_" + generateUuid()); - ARROW_ASSIGN_OR_RAISE(auto file_info, fs->GetFileInfo(filePath)); - if (file_info.type() == arrow::fs::FileType::NotFound) { - int fd = open(filePath.c_str(), O_CREAT | O_EXCL | O_RDWR, 0666); + filePath = parentPath / ("temp-shuffle-" + generateUuid()); + if (!std::filesystem::exists(filePath)) { + auto fd = open(filePath.c_str(), O_CREAT | O_EXCL | O_RDWR, 0666); if (fd < 0) { if (errno != EEXIST) { - return arrow::Status::IOError("Failed to open local file " + filePath + ", Reason: " + strerror(errno)); + return arrow::Status::IOError( + "Failed to open local file " + filePath.string() + ", Reason: " + strerror(errno)); } } else { exist = false; diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index c4e2409d2da0..2e5ff58b6ebf 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -18,13 +18,13 @@ #pragma once #include -#include -#include -#include #include #include #include + #include +#include + #include "utils/Compression.h" namespace gluten { @@ -36,9 +36,7 @@ static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBuf static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); static const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; -std::string generateUuid(); - -std::string getSpilledShuffleFileDir(const std::string& configuredDir, int32_t subDirId); +std::string getShuffleSpillDir(const std::string& configuredDir, int32_t subDirId); arrow::Result createTempShuffleFile(const std::string& dir); @@ -72,4 +70,39 @@ arrow::Result> makeUncompressedRecordBatch( std::shared_ptr zeroLengthNullBuffer(); +// MmapFileStream is used to optimize sequential file reading. It uses madvise +// to prefetch and release memory timely. +class MmapFileStream : public arrow::io::InputStream { + public: + MmapFileStream(arrow::internal::FileDescriptor fd, uint8_t* data, int64_t size, uint64_t prefetchSize); + + static arrow::Result> open(const std::string& path, uint64_t prefetchSize = 0); + + arrow::Result Tell() const override; + + arrow::Status Close() override; + + arrow::Result Read(int64_t nbytes, void* out) override; + + arrow::Result> Read(int64_t nbytes) override; + + bool closed() const override; + + private: + arrow::Result actualReadSize(int64_t nbytes); + + void advance(int64_t length); + + void willNeed(int64_t length); + + // Page-aligned prefetch size + const int64_t prefetchSize_; + arrow::internal::FileDescriptor fd_; + uint8_t* data_ = nullptr; + int64_t size_; + int64_t pos_ = 0; + int64_t posFetch_ = 0; + int64_t posRetain_ = 0; +}; + } // namespace gluten diff --git a/cpp/core/utils/Metrics.h b/cpp/core/utils/Metrics.h index 698e97f3dce6..8da13bc91635 100644 --- a/cpp/core/utils/Metrics.h +++ b/cpp/core/utils/Metrics.h @@ -95,8 +95,8 @@ struct Metrics { Metrics& operator=(Metrics&&) = delete; long* get(TYPE type) { - assert((int)type >= (int)kBegin && (int)type < (int)kEnd); - auto offset = ((int)type - (int)kBegin) * numMetrics; + assert(static_cast(type) >= static_cast(kBegin) && static_cast(type) < static_cast(kEnd)); + auto offset = (static_cast(type) - static_cast(kBegin)) * numMetrics; return &arrayRawPtr[offset]; } }; diff --git a/cpp/core/utils/Registry.h b/cpp/core/utils/Registry.h index e50eb6763d72..a325a05e5a85 100644 --- a/cpp/core/utils/Registry.h +++ b/cpp/core/utils/Registry.h @@ -29,19 +29,19 @@ class Registry { public: void registerObj(const std::string& kind, T t) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) == map_.end(), "Already registered for " + kind); + GLUTEN_CHECK(map_.find(kind) == map_.end(), "Required object already registered for " + kind); map_[kind] = std::move(t); } T& get(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "Not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Required object not registered for " + kind); return map_[kind]; } bool unregisterObj(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "Not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Required object not registered for " + kind); return map_.erase(kind); } diff --git a/cpp/core/utils/StringUtil.cc b/cpp/core/utils/StringUtil.cc index fc6ebb83c0bd..505e1972f8d0 100644 --- a/cpp/core/utils/StringUtil.cc +++ b/cpp/core/utils/StringUtil.cc @@ -17,13 +17,16 @@ #include #include -#include #include +#include +#include + #include "Exception.h" #include "StringUtil.h" -std::vector gluten::splitByDelim(const std::string& s, const char delimiter) { +namespace gluten { +std::vector splitByDelim(const std::string& s, const char delimiter) { if (s.empty()) { return {}; } @@ -41,7 +44,7 @@ std::vector gluten::splitByDelim(const std::string& s, const char d return result; } -std::vector gluten::splitPaths(const std::string& s, bool checkExists) { +std::vector splitPaths(const std::string& s, bool checkExists) { if (s.empty()) { return {}; } @@ -61,3 +64,10 @@ std::vector gluten::splitPaths(const std::string& s, bool checkExis } return paths; } + +std::string generateUuid() { + boost::uuids::random_generator generator; + return boost::uuids::to_string(generator()); +} + +} // namespace gluten diff --git a/cpp/core/utils/StringUtil.h b/cpp/core/utils/StringUtil.h index 8880229616a5..3030651aa391 100644 --- a/cpp/core/utils/StringUtil.h +++ b/cpp/core/utils/StringUtil.h @@ -23,4 +23,6 @@ std::vector splitByDelim(const std::string& s, const char delimiter std::vector splitPaths(const std::string& s, bool checkExists = false); +std::string generateUuid(); + } // namespace gluten diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index b16de0032a04..4dc5ec0aff43 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -23,12 +23,6 @@ include(GNUInstallDirs) include(CheckCXXCompilerFlag) include(FindPackageHandleStandardArgs) -set(CMAKE_CXX_FLAGS - "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attributes") -if(NOT ${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-class-memaccess") -endif() - set(SYSTEM_LIB_PATH "/usr/lib" CACHE PATH "System Lib dir") @@ -62,18 +56,6 @@ if(NOT DEFINED VELOX_HOME) message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") endif() -# Keep same compile option with Velox. -execute_process( - COMMAND - bash -c - "( source ${VELOX_HOME}/scripts/setup-helper-functions.sh && echo -n $(get_cxx_flags $ENV{CPU_TARGET}))" - OUTPUT_VARIABLE SCRIPT_CXX_FLAGS - RESULT_VARIABLE COMMAND_STATUS) -if(COMMAND_STATUS EQUAL "1") - message(FATAL_ERROR "Unable to determine compiler flags!") -endif() -set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SCRIPT_CXX_FLAGS}") - message("Velox module final CMAKE_CXX_FLAGS=${CMAKE_CXX_FLAGS}") # User can specify VELOX_BUILD_PATH, if Velox are built elsewhere. @@ -130,61 +112,72 @@ macro(find_re2) endmacro() macro(find_awssdk) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(AWSSDK REQUIRED COMPONENTS s3;identity-management) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() macro(find_gcssdk) - set(CMAKE_FIND_LIBRARY_SUFFIXES ".so") - find_package(google_cloud_cpp_storage REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) + set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") + find_package(google_cloud_cpp_storage CONFIG 2.22.0 REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() macro(find_azure) find_package(CURL REQUIRED) find_package(LibXml2 REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES_BCK ${CMAKE_FIND_LIBRARY_SUFFIXES}) set(CMAKE_FIND_LIBRARY_SUFFIXES ".a") find_package(azure-storage-blobs-cpp CONFIG REQUIRED) find_package(azure-storage-files-datalake-cpp CONFIG REQUIRED) + find_package(azure-identity-cpp CONFIG REQUIRED) + set(CMAKE_FIND_LIBRARY_SUFFIXES ${CMAKE_FIND_LIBRARY_SUFFIXES_BCK}) endmacro() # Build Velox backend. set(VELOX_SRCS compute/VeloxBackend.cc compute/VeloxRuntime.cc - compute/WholeStageResultIterator.cc compute/VeloxPlanConverter.cc + compute/WholeStageResultIterator.cc compute/iceberg/IcebergPlanConverter.cc - jni/VeloxJniWrapper.cc jni/JniFileSystem.cc jni/JniUdf.cc + jni/VeloxJniWrapper.cc memory/BufferOutputStream.cc memory/VeloxColumnarBatch.cc memory/VeloxMemoryManager.cc operators/functions/RegistrationAllFunctions.cc operators/functions/RowConstructorWithNull.cc - operators/serializer/VeloxColumnarToRowConverter.cc + operators/functions/SparkExprToSubfieldFilterParser.cc + operators/reader/FileReaderIterator.cc + operators/reader/ParquetReaderIterator.cc operators/serializer/VeloxColumnarBatchSerializer.cc + operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxRowToColumnarConverter.cc + operators/writer/VeloxArrowWriter.cc operators/writer/VeloxParquetDataSource.cc + shuffle/VeloxHashShuffleWriter.cc + shuffle/VeloxRssSortShuffleWriter.cc shuffle/VeloxShuffleReader.cc shuffle/VeloxShuffleWriter.cc - shuffle/VeloxHashShuffleWriter.cc shuffle/VeloxSortShuffleWriter.cc - shuffle/VeloxRssSortShuffleWriter.cc + substrait/SubstraitExtensionCollector.cc substrait/SubstraitParser.cc substrait/SubstraitToVeloxExpr.cc substrait/SubstraitToVeloxPlan.cc substrait/SubstraitToVeloxPlanValidator.cc substrait/VariantToVectorConverter.cc - substrait/SubstraitExtensionCollector.cc substrait/VeloxSubstraitSignature.cc substrait/VeloxToSubstraitExpr.cc substrait/VeloxToSubstraitPlan.cc substrait/VeloxToSubstraitType.cc udf/UdfLoader.cc - utils/VeloxArrowUtils.cc - utils/ConfigExtractor.cc utils/Common.cc + utils/ConfigExtractor.cc + utils/VeloxArrowUtils.cc utils/VeloxBatchResizer.cc) if(ENABLE_S3) @@ -197,8 +190,8 @@ endif() add_library(velox SHARED ${VELOX_SRCS}) -if(ENABLE_GLUTEN_VCPKG) - # Hide symbols of static dependencies +if(ENABLE_GLUTEN_VCPKG AND NOT CMAKE_SYSTEM_NAME MATCHES "Darwin") + # Hide some symbols to avoid conflict. target_link_options( velox PRIVATE -Wl,--version-script=${CMAKE_CURRENT_SOURCE_DIR}/symbols.map) endif() @@ -220,14 +213,14 @@ set_target_properties(velox PROPERTIES LIBRARY_OUTPUT_DIRECTORY find_package(Folly REQUIRED CONFIG) if(ENABLE_JEMALLOC_STATS) - include(Findjemalloc_pic) + include(Findjemalloc) find_jemalloc() if(JEMALLOC_NOT_FOUND) - include(Buildjemalloc_pic) + include(Buildjemalloc) build_jemalloc() endif() add_definitions(-DENABLE_JEMALLOC_STATS) - target_link_libraries(velox PUBLIC jemalloc::libjemalloc) + target_link_libraries(velox PUBLIC jemalloc::jemalloc) endif() target_link_libraries(velox PUBLIC gluten) @@ -258,12 +251,15 @@ if(BUILD_TESTS) import_library( facebook::velox::exec_test_lib ${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_exec_test_lib.a) + import_library(facebook::velox::cursor + ${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_cursor.a) target_link_libraries( facebook::velox::exec_test_lib INTERFACE facebook::velox::vector_test_lib facebook::velox::dwio_common_test facebook::velox::file_test_utils facebook::velox::temp_path) target_link_libraries(velox PUBLIC facebook::velox::exec_test_lib) + target_link_libraries(velox PUBLIC facebook::velox::cursor) endif() target_link_libraries(velox PUBLIC facebook::velox) @@ -342,6 +338,7 @@ if(ENABLE_ABFS) find_azure() target_link_libraries(velox PUBLIC Azure::azure-storage-blobs) target_link_libraries(velox PUBLIC Azure::azure-storage-files-datalake) + target_link_libraries(velox PUBLIC Azure::azure-identity) endif() if(BUILD_EXAMPLES) diff --git a/cpp/velox/benchmarks/CMakeLists.txt b/cpp/velox/benchmarks/CMakeLists.txt index 1aa199b13696..6b2cda358c06 100644 --- a/cpp/velox/benchmarks/CMakeLists.txt +++ b/cpp/velox/benchmarks/CMakeLists.txt @@ -15,8 +15,7 @@ find_arrow_lib(${PARQUET_LIB_NAME}) -set(VELOX_BENCHMARK_COMMON_SRCS common/FileReaderIterator.cc - common/BenchmarkUtils.cc) +set(VELOX_BENCHMARK_COMMON_SRCS common/BenchmarkUtils.cc) add_library(velox_benchmark_common STATIC ${VELOX_BENCHMARK_COMMON_SRCS}) target_include_directories( velox_benchmark_common PUBLIC ${CMAKE_SOURCE_DIR}/velox @@ -38,7 +37,3 @@ add_velox_benchmark(columnar_to_row_benchmark ColumnarToRowBenchmark.cc) add_velox_benchmark(parquet_write_benchmark ParquetWriteBenchmark.cc) add_velox_benchmark(plan_validator_util PlanValidatorUtil.cc) - -if(ENABLE_ORC) - add_velox_benchmark(orc_converter exec/OrcConverter.cc) -endif() diff --git a/cpp/velox/benchmarks/GenericBenchmark.cc b/cpp/velox/benchmarks/GenericBenchmark.cc index e42aed9f219f..dcb64d7d18d0 100644 --- a/cpp/velox/benchmarks/GenericBenchmark.cc +++ b/cpp/velox/benchmarks/GenericBenchmark.cc @@ -25,12 +25,12 @@ #include #include "benchmarks/common/BenchmarkUtils.h" -#include "benchmarks/common/FileReaderIterator.h" #include "compute/VeloxBackend.h" -#include "compute/VeloxPlanConverter.h" #include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" #include "config/VeloxConfig.h" +#include "operators/reader/FileReaderIterator.h" +#include "operators/writer/VeloxArrowWriter.h" #include "shuffle/LocalPartitionWriter.h" #include "shuffle/VeloxShuffleWriter.h" #include "shuffle/rss/RssPartitionWriter.h" @@ -45,7 +45,6 @@ using namespace gluten; namespace { -DEFINE_bool(run_example, false, "Run the example and exit."); DEFINE_bool(print_result, true, "Print result for execution"); DEFINE_string(save_output, "", "Path to parquet file for saving the task output iterator"); DEFINE_bool(with_shuffle, false, "Add shuffle split at end."); @@ -106,6 +105,50 @@ void setUpBenchmark(::benchmark::internal::Benchmark* bm) { } } +std::string generateUniqueSubdir(const std::string& parent, const std::string& prefix = "") { + auto path = std::filesystem::path(parent) / (prefix + generateUuid()); + std::error_code ec{}; + while (!std::filesystem::create_directories(path, ec)) { + if (ec) { + LOG(ERROR) << fmt::format("Failed to created spill directory: {}, error code: {}", path, ec.message()); + std::exit(EXIT_FAILURE); + } + path = std::filesystem::path(parent) / (prefix + generateUuid()); + } + return path; +} + +std::vector createLocalDirs() { + static const std::string kBenchmarkDirPrefix = "generic-benchmark-"; + std::vector localDirs; + + auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); + // Check if local dirs are set from env. + if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { + auto joinedDirs = std::string(joinedDirsC); + auto dirs = gluten::splitPaths(joinedDirs); + for (const auto& dir : dirs) { + localDirs.push_back(generateUniqueSubdir(dir, kBenchmarkDirPrefix)); + } + } else { + // Otherwise create 1 temp dir. + localDirs.push_back(generateUniqueSubdir(std::filesystem::temp_directory_path(), kBenchmarkDirPrefix)); + } + return localDirs; +} + +void cleanupLocalDirs(const std::vector& localDirs) { + for (const auto& localDir : localDirs) { + std::error_code ec; + std::filesystem::remove_all(localDir, ec); + if (ec) { + LOG(WARNING) << fmt::format("Failed to remove directory: {}, error message: {}", localDir, ec.message()); + } else { + LOG(INFO) << "Removed local dir: " << localDir; + } + } +} + PartitionWriterOptions createPartitionWriterOptions() { PartitionWriterOptions partitionWriterOptions{}; // Disable writer's merge. @@ -204,11 +247,10 @@ void runShuffle( const std::shared_ptr& resultIter, WriterMetrics& writerMetrics, ReaderMetrics& readerMetrics, - bool readAfterWrite) { - std::string dataFile; - std::vector localDirs; - bool isFromEnv; - GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(dataFile, localDirs, isFromEnv)); + bool readAfterWrite, + const std::vector& localDirs, + const std::string& dataFileDir) { + GLUTEN_ASSIGN_OR_THROW(auto dataFile, gluten::createTempShuffleFile(dataFileDir)); auto partitionWriterOptions = createPartitionWriterOptions(); auto partitionWriter = createPartitionWriter(runtime, partitionWriterOptions, dataFile, localDirs); @@ -252,8 +294,12 @@ void runShuffle( readerMetrics.decompressTime = reader->getDecompressTime(); readerMetrics.deserializeTime = reader->getDeserializeTime(); } - // Cleanup shuffle outputs - cleanupShuffleOutput(dataFile, localDirs, isFromEnv); + + if (std::filesystem::remove(dataFile)) { + LOG(INFO) << "Removed shuffle data file: " << dataFile; + } else { + LOG(WARNING) << "Failed to remove shuffle data file. File does not exist: " << dataFile; + } } void updateBenchmarkMetrics( @@ -292,7 +338,6 @@ void updateBenchmarkMetrics( writerMetrics.bytesWritten, benchmark::Counter::kAvgIterations, benchmark::Counter::OneK::kIs1024); } } - } // namespace using RuntimeFactory = std::function; @@ -301,6 +346,7 @@ auto BM_Generic = [](::benchmark::State& state, const std::string& planFile, const std::vector& splitFiles, const std::vector& dataFiles, + const std::vector& localDirs, RuntimeFactory runtimeFactory, FileReaderType readerType) { setCpu(state); @@ -316,6 +362,19 @@ auto BM_Generic = [](::benchmark::State& state, splits.push_back(getPlanFromFile("ReadRel.LocalFiles", splitFile)); } + const auto tid = std::hash{}(std::this_thread::get_id()); + const auto spillDirIndex = tid % localDirs.size(); + const auto veloxSpillDir = generateUniqueSubdir(std::filesystem::path(localDirs[spillDirIndex]) / "gluten-spill"); + + std::vector shuffleSpillDirs; + std::transform(localDirs.begin(), localDirs.end(), std::back_inserter(shuffleSpillDirs), [](const auto& dir) { + auto path = std::filesystem::path(dir) / "shuffle-write"; + return path; + }); + // Use a different directory for data file. + const auto dataFileDir = gluten::getShuffleSpillDir( + shuffleSpillDirs[(spillDirIndex + 1) % localDirs.size()], state.thread_index() % gluten::kDefaultNumSubDirs); + WriterMetrics writerMetrics{}; ReaderMetrics readerMetrics{}; int64_t readInputTime = 0; @@ -328,7 +387,8 @@ auto BM_Generic = [](::benchmark::State& state, std::vector inputItersRaw; if (!dataFiles.empty()) { for (const auto& input : dataFiles) { - inputIters.push_back(getInputIteratorFromFileReader(input, readerType)); + inputIters.push_back(FileReaderIterator::getInputIteratorFromFileReader( + readerType, input, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get())); } std::transform( inputIters.begin(), @@ -343,11 +403,13 @@ auto BM_Generic = [](::benchmark::State& state, for (auto& split : splits) { runtime->parseSplitInfo(reinterpret_cast(split.data()), split.size(), std::nullopt); } - auto resultIter = runtime->createResultIterator("/tmp/test-spill", std::move(inputIters), runtime->getConfMap()); + + auto resultIter = runtime->createResultIterator(veloxSpillDir, std::move(inputIters), runtime->getConfMap()); listenerPtr->setIterator(resultIter.get()); if (FLAGS_with_shuffle) { - runShuffle(runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, false); + runShuffle( + runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, false, shuffleSpillDirs, dataFileDir); } else { // May write the output into file. auto veloxPlan = dynamic_cast(runtime)->getVeloxPlan(); @@ -355,10 +417,11 @@ auto BM_Generic = [](::benchmark::State& state, ArrowSchema cSchema; toArrowSchema(veloxPlan->outputType(), runtime->memoryManager()->getLeafMemoryPool().get(), &cSchema); GLUTEN_ASSIGN_OR_THROW(auto outputSchema, arrow::ImportSchema(&cSchema)); - ArrowWriter writer{FLAGS_save_output}; + auto writer = std::make_shared( + FLAGS_save_output, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get()); state.PauseTiming(); if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.initWriter(*(outputSchema.get()))); + GLUTEN_THROW_NOT_OK(writer->initWriter(*(outputSchema.get()))); } state.ResumeTiming(); @@ -374,13 +437,13 @@ auto BM_Generic = [](::benchmark::State& state, LOG(WARNING) << maybeBatch.ValueOrDie()->ToString(); } if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.writeInBatches(maybeBatch.ValueOrDie())); + GLUTEN_THROW_NOT_OK(writer->writeInBatches(maybeBatch.ValueOrDie())); } } state.PauseTiming(); if (!FLAGS_save_output.empty()) { - GLUTEN_THROW_NOT_OK(writer.closeWriter()); + GLUTEN_THROW_NOT_OK(writer->closeWriter()); } state.ResumeTiming(); } @@ -405,6 +468,7 @@ auto BM_Generic = [](::benchmark::State& state, auto BM_ShuffleWriteRead = [](::benchmark::State& state, const std::string& inputFile, + const std::vector& localDirs, RuntimeFactory runtimeFactory, FileReaderType readerType) { setCpu(state); @@ -414,6 +478,10 @@ auto BM_ShuffleWriteRead = [](::benchmark::State& state, auto* memoryManager = MemoryManager::create(kVeloxBackendKind, std::move(listener)); auto runtime = runtimeFactory(memoryManager); + const size_t dirIndex = std::hash{}(std::this_thread::get_id()) % localDirs.size(); + const auto dataFileDir = + gluten::getShuffleSpillDir(localDirs[dirIndex], state.thread_index() % gluten::kDefaultNumSubDirs); + WriterMetrics writerMetrics{}; ReaderMetrics readerMetrics{}; int64_t readInputTime = 0; @@ -421,8 +489,17 @@ auto BM_ShuffleWriteRead = [](::benchmark::State& state, { ScopedTimer timer(&elapsedTime); for (auto _ : state) { - auto resultIter = getInputIteratorFromFileReader(inputFile, readerType); - runShuffle(runtime, listenerPtr, resultIter, writerMetrics, readerMetrics, FLAGS_run_shuffle_read); + auto resultIter = FileReaderIterator::getInputIteratorFromFileReader( + readerType, inputFile, FLAGS_batch_size, runtime->memoryManager()->getLeafMemoryPool().get()); + runShuffle( + runtime, + listenerPtr, + resultIter, + writerMetrics, + readerMetrics, + FLAGS_run_shuffle_read, + localDirs, + dataFileDir); auto reader = static_cast(resultIter->getInputIter()); readInputTime += reader->getCollectBatchTime(); @@ -516,19 +593,7 @@ int main(int argc, char** argv) { std::vector splitFiles{}; std::vector dataFiles{}; - if (FLAGS_run_example) { - LOG(WARNING) << "Running example..."; - dataFiles.resize(2); - try { - substraitJsonFile = getGeneratedFilePath("example.json"); - dataFiles[0] = getGeneratedFilePath("example_orders"); - dataFiles[1] = getGeneratedFilePath("example_lineitem"); - } catch (const std::exception& e) { - LOG(ERROR) << "Failed to run example. " << e.what(); - ::benchmark::Shutdown(); - std::exit(EXIT_FAILURE); - } - } else if (FLAGS_run_shuffle) { + if (FLAGS_run_shuffle) { std::string errorMsg{}; if (FLAGS_data.empty()) { errorMsg = "Missing '--split' or '--data' option."; @@ -600,23 +665,31 @@ int main(int argc, char** argv) { return dynamic_cast(Runtime::create(kVeloxBackendKind, memoryManager, sessionConf)); }; -#define GENERIC_BENCHMARK(READER_TYPE) \ - do { \ - auto* bm = \ - ::benchmark::RegisterBenchmark( \ - "GenericBenchmark", BM_Generic, substraitJsonFile, splitFiles, dataFiles, runtimeFactory, READER_TYPE) \ - ->MeasureProcessCPUTime() \ - ->UseRealTime(); \ - setUpBenchmark(bm); \ + const auto localDirs = createLocalDirs(); + +#define GENERIC_BENCHMARK(READER_TYPE) \ + do { \ + auto* bm = ::benchmark::RegisterBenchmark( \ + "GenericBenchmark", \ + BM_Generic, \ + substraitJsonFile, \ + splitFiles, \ + dataFiles, \ + localDirs, \ + runtimeFactory, \ + READER_TYPE) \ + ->MeasureProcessCPUTime() \ + ->UseRealTime(); \ + setUpBenchmark(bm); \ } while (0) -#define SHUFFLE_WRITE_READ_BENCHMARK(READER_TYPE) \ - do { \ - auto* bm = ::benchmark::RegisterBenchmark( \ - "ShuffleWriteRead", BM_ShuffleWriteRead, dataFiles[0], runtimeFactory, READER_TYPE) \ - ->MeasureProcessCPUTime() \ - ->UseRealTime(); \ - setUpBenchmark(bm); \ +#define SHUFFLE_WRITE_READ_BENCHMARK(READER_TYPE) \ + do { \ + auto* bm = ::benchmark::RegisterBenchmark( \ + "ShuffleWriteRead", BM_ShuffleWriteRead, dataFiles[0], localDirs, runtimeFactory, READER_TYPE) \ + ->MeasureProcessCPUTime() \ + ->UseRealTime(); \ + setUpBenchmark(bm); \ } while (0) if (dataFiles.empty()) { @@ -642,5 +715,7 @@ int main(int argc, char** argv) { gluten::VeloxBackend::get()->tearDown(); + cleanupLocalDirs(localDirs); + return 0; } diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.cc b/cpp/velox/benchmarks/common/BenchmarkUtils.cc index 345f9da8e16d..d0a26ff07fbd 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.cc +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.cc @@ -23,9 +23,6 @@ #include "utils/StringUtil.h" #include "velox/dwio/common/Options.h" -using namespace facebook; -namespace fs = std::filesystem; - DEFINE_int64(batch_size, 4096, "To set velox::core::QueryConfig::kPreferredOutputBatchSize."); DEFINE_int32(cpu, -1, "Run benchmark on specific CPU"); DEFINE_int32(threads, 1, "The number of threads to run this benchmark"); @@ -34,7 +31,7 @@ DEFINE_int32(iterations, 1, "The number of iterations to run this benchmark"); namespace gluten { namespace { std::unordered_map bmConfMap = defaultConf(); -} +} // namespace std::unordered_map defaultConf() { return { @@ -60,13 +57,13 @@ std::string getPlanFromFile(const std::string& type, const std::string& filePath return gluten::substraitFromJsonToPb(type, msgData); } -velox::dwio::common::FileFormat getFileFormat(const std::string& fileFormat) { +facebook::velox::dwio::common::FileFormat getFileFormat(const std::string& fileFormat) { if (fileFormat.compare("orc") == 0) { - return velox::dwio::common::FileFormat::ORC; + return facebook::velox::dwio::common::FileFormat::ORC; } else if (fileFormat.compare("parquet") == 0) { - return velox::dwio::common::FileFormat::PARQUET; + return facebook::velox::dwio::common::FileFormat::PARQUET; } else { - return velox::dwio::common::FileFormat::UNKNOWN; + return facebook::velox::dwio::common::FileFormat::UNKNOWN; } } @@ -84,7 +81,7 @@ std::shared_ptr getSplitInfos(const std::string& datasetPath, if (endsWith(singleFilePath, "." + fileFormat)) { auto fileAbsolutePath = datasetPath + singleFilePath; scanInfo->starts.emplace_back(0); - scanInfo->lengths.emplace_back(fs::file_size(fileAbsolutePath)); + scanInfo->lengths.emplace_back(std::filesystem::file_size(fileAbsolutePath)); scanInfo->paths.emplace_back("file://" + fileAbsolutePath); } } else { @@ -102,7 +99,7 @@ std::shared_ptr getSplitInfosFromFile(const std::string& file // Set split start, length, and path to scan info. scanInfo->starts.emplace_back(0); - scanInfo->lengths.emplace_back(fs::file_size(fileName)); + scanInfo->lengths.emplace_back(std::filesystem::file_size(fileName)); scanInfo->paths.emplace_back("file://" + fileName); return scanInfo; @@ -125,78 +122,25 @@ bool endsWith(const std::string& data, const std::string& suffix) { return data.find(suffix, data.size() - suffix.size()) != std::string::npos; } -#if 0 -std::shared_ptr createReader(const std::string& path) { - std::unique_ptr parquetReader; - std::shared_ptr recordBatchReader; - parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); - - GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( - arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path), properties, &parquetReader)); - GLUTEN_THROW_NOT_OK( - parquetReader->GetRecordBatchReader(arrow::internal::Iota(parquetReader->num_row_groups()), &recordBatchReader)); - return recordBatchReader; -} -#endif - -void setCpu(uint32_t cpuindex) { +void setCpu(uint32_t cpuIndex) { static const auto kTotalCores = std::thread::hardware_concurrency(); - cpuindex = cpuindex % kTotalCores; + cpuIndex = cpuIndex % kTotalCores; cpu_set_t cs; CPU_ZERO(&cs); - CPU_SET(cpuindex, &cs); + CPU_SET(cpuIndex, &cs); if (sched_setaffinity(0, sizeof(cs), &cs) == -1) { - LOG(WARNING) << "Error binding CPU " << std::to_string(cpuindex); - exit(EXIT_FAILURE); - } -} - -arrow::Status -setLocalDirsAndDataFileFromEnv(std::string& dataFile, std::vector& localDirs, bool& isFromEnv) { - auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); - if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { - isFromEnv = true; - // Set local dirs. - auto joinedDirs = std::string(joinedDirsC); - // Split local dirs and use thread id to choose one directory for data file. - auto dirs = gluten::splitPaths(joinedDirs); - for (const auto& dir : dirs) { - localDirs.push_back(arrow::fs::internal::ConcatAbstractPath(dir, "temp_shuffle_" + generateUuid())); - std::filesystem::create_directory(localDirs.back()); - } - size_t id = std::hash{}(std::this_thread::get_id()) % localDirs.size(); - ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs[id])); - } else { - isFromEnv = false; - // Otherwise create 1 temp dir and data file. - static const std::string kBenchmarkDirsPrefix = "columnar-shuffle-benchmark-"; - { - // Because tmpDir will be deleted in the dtor, allow it to be deleted upon exiting the block and then recreate it - // in createTempShuffleFile. - ARROW_ASSIGN_OR_RAISE(auto tmpDir, arrow::internal::TemporaryDir::Make(kBenchmarkDirsPrefix)) - localDirs.push_back(tmpDir->path().ToString()); - } - ARROW_ASSIGN_OR_RAISE(dataFile, gluten::createTempShuffleFile(localDirs.back())); - } - return arrow::Status::OK(); -} - -void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv) { - std::filesystem::remove(dataFile); - for (auto& localDir : localDirs) { - if (std::filesystem::is_empty(localDir)) { - std::filesystem::remove(localDir); - } + LOG(WARNING) << "Error binding CPU " << std::to_string(cpuIndex); + std::exit(EXIT_FAILURE); } } void BenchmarkAllocationListener::allocationChanged(int64_t diff) { - if (usedBytes_ + diff >= limit_) { + if (diff > 0 && usedBytes_ + diff >= limit_) { LOG(INFO) << fmt::format( "reach hard limit {} when need {}, current used {}.", - velox::succinctBytes(limit_), - velox::succinctBytes(diff), - velox::succinctBytes(usedBytes_)); + facebook::velox::succinctBytes(limit_), + facebook::velox::succinctBytes(diff), + facebook::velox::succinctBytes(usedBytes_)); auto neededBytes = usedBytes_ + diff - limit_; int64_t spilledBytes = 0; if (iterator_) { @@ -207,7 +151,7 @@ void BenchmarkAllocationListener::allocationChanged(int64_t diff) { GLUTEN_THROW_NOT_OK(shuffleWriter_->reclaimFixedSize(neededBytes - spilledBytes, &reclaimed)); spilledBytes += reclaimed; } - LOG(INFO) << fmt::format("spill finish, got {}.", velox::succinctBytes(spilledBytes)); + LOG(INFO) << fmt::format("spill finish, got {}.", facebook::velox::succinctBytes(spilledBytes)); } else { usedBytes_ += diff; } diff --git a/cpp/velox/benchmarks/common/BenchmarkUtils.h b/cpp/velox/benchmarks/common/BenchmarkUtils.h index 0108f1d44838..de3df96f89e0 100644 --- a/cpp/velox/benchmarks/common/BenchmarkUtils.h +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.h @@ -95,12 +95,7 @@ inline std::shared_ptr convertBatch(std::shared_ptr& localDirs, bool& isFromEnv); - -void cleanupShuffleOutput(const std::string& dataFile, const std::vector& localDirs, bool isFromEnv); +void setCpu(uint32_t cpuIndex); class BenchmarkAllocationListener final : public gluten::AllocationListener { public: @@ -118,7 +113,7 @@ class BenchmarkAllocationListener final : public gluten::AllocationListener { private: uint64_t usedBytes_{0L}; - uint64_t limit_{0L}; + const uint64_t limit_{0L}; gluten::ResultIterator* iterator_{nullptr}; gluten::ShuffleWriter* shuffleWriter_{nullptr}; }; diff --git a/cpp/velox/benchmarks/common/OrcReaderIterator.h b/cpp/velox/benchmarks/common/OrcReaderIterator.h deleted file mode 100644 index f8c9f44b2008..000000000000 --- a/cpp/velox/benchmarks/common/OrcReaderIterator.h +++ /dev/null @@ -1,107 +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. - */ -#pragma once - -#include -#include "benchmarks/common/FileReaderIterator.h" - -namespace gluten { - -class OrcReaderIterator : public FileReaderIterator { - public: - explicit OrcReaderIterator(const std::string& path) : FileReaderIterator(path) {} - - void createReader() override { - // Open File - auto input = arrow::io::ReadableFile::Open(path_); - GLUTEN_THROW_NOT_OK(input); - - // Open ORC File Reader - auto maybeReader = arrow::adapters::orc::ORCFileReader::Open(*input, arrow::default_memory_pool()); - GLUTEN_THROW_NOT_OK(maybeReader); - fileReader_.reset((*maybeReader).release()); - - // get record batch Reader - auto recordBatchReader = fileReader_->GetRecordBatchReader(4096, std::vector()); - GLUTEN_THROW_NOT_OK(recordBatchReader); - recordBatchReader_ = *recordBatchReader; - } - - std::shared_ptr getSchema() override { - auto schema = fileReader_->ReadSchema(); - GLUTEN_THROW_NOT_OK(schema); - return *schema; - } - - protected: - std::unique_ptr fileReader_; - std::shared_ptr recordBatchReader_; -}; - -class OrcStreamReaderIterator final : public OrcReaderIterator { - public: - explicit OrcStreamReaderIterator(const std::string& path) : OrcReaderIterator(path) { - createReader(); - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DLOG(INFO) << "OrcStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -class OrcBufferedReaderIterator final : public OrcReaderIterator { - public: - explicit OrcBufferedReaderIterator(const std::string& path) : OrcReaderIterator(path) { - createReader(); - collectBatches(); - iter_ = batches_.begin(); - DLOG(INFO) << "OrcBufferedReaderIterator open file: " << path; - DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); - if (iter_ != batches_.cend()) { - DLOG(INFO) << "columns: " << (*iter_)->num_columns(); - DLOG(INFO) << "rows: " << (*iter_)->num_rows(); - } - } - - std::shared_ptr next() override { - if (iter_ == batches_.cend()) { - return nullptr; - } - return convertBatch(std::make_shared(*iter_++)); - } - - private: - void collectBatches() { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); - auto endTime = std::chrono::steady_clock::now(); - collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); - } - - arrow::RecordBatchVector batches_; - std::vector>::const_iterator iter_; -}; - -} // namespace gluten \ No newline at end of file diff --git a/cpp/velox/benchmarks/common/ParquetReaderIterator.h b/cpp/velox/benchmarks/common/ParquetReaderIterator.h deleted file mode 100644 index 20652ee27dc0..000000000000 --- a/cpp/velox/benchmarks/common/ParquetReaderIterator.h +++ /dev/null @@ -1,104 +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. - */ - -#include "benchmarks/common/BenchmarkUtils.h" -#include "benchmarks/common/FileReaderIterator.h" -#include "utils/Macros.h" - -#include - -namespace gluten { - -class ParquetReaderIterator : public FileReaderIterator { - public: - explicit ParquetReaderIterator(const std::string& path) : FileReaderIterator(path) {} - - void createReader() { - parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); - properties.set_batch_size(FLAGS_batch_size); - GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( - arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path_), properties, &fileReader_)); - GLUTEN_THROW_NOT_OK( - fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); - - auto schema = recordBatchReader_->schema(); - LOG(INFO) << "schema:\n" << schema->ToString(); - } - - std::shared_ptr getSchema() override { - return recordBatchReader_->schema(); - } - - protected: - std::unique_ptr fileReader_; - std::shared_ptr recordBatchReader_; -}; - -class ParquetStreamReaderIterator final : public ParquetReaderIterator { - public: - explicit ParquetStreamReaderIterator(const std::string& path) : ParquetReaderIterator(path) { - createReader(); - DLOG(INFO) << "ParquetStreamReaderIterator open file: " << path; - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DLOG(INFO) << "ParquetStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -class ParquetBufferedReaderIterator final : public ParquetReaderIterator { - public: - explicit ParquetBufferedReaderIterator(const std::string& path) : ParquetReaderIterator(path) { - createReader(); - collectBatches(); - iter_ = batches_.begin(); - DLOG(INFO) << "ParquetBufferedReaderIterator open file: " << path; - DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); - if (iter_ != batches_.cend()) { - DLOG(INFO) << "columns: " << (*iter_)->num_columns(); - DLOG(INFO) << "rows: " << (*iter_)->num_rows(); - } - } - - std::shared_ptr next() override { - if (iter_ == batches_.cend()) { - return nullptr; - } - return convertBatch(std::make_shared(*iter_++)); - } - - private: - void collectBatches() { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); - auto endTime = std::chrono::steady_clock::now(); - collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); - } - - arrow::RecordBatchVector batches_; - std::vector>::const_iterator iter_; -}; - -} // namespace gluten diff --git a/cpp/velox/benchmarks/exec/OrcConverter.cc b/cpp/velox/benchmarks/exec/OrcConverter.cc index b421ecca3b37..888cf27c35fe 100644 --- a/cpp/velox/benchmarks/exec/OrcConverter.cc +++ b/cpp/velox/benchmarks/exec/OrcConverter.cc @@ -16,7 +16,7 @@ */ #include -#include "benchmarks/common/ParquetReaderIterator.h" +#include "operators/reader/ParquetReaderIterator.h" namespace gluten { diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index d39b0902c250..c453b9981fab 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -33,6 +33,7 @@ #include "compute/VeloxRuntime.h" #include "config/VeloxConfig.h" #include "jni/JniFileSystem.h" +#include "operators/functions/SparkExprToSubfieldFilterParser.h" #include "udf/UdfLoader.h" #include "utils/Exception.h" #include "velox/common/caching/SsdCache.h" @@ -40,7 +41,7 @@ #include "velox/connectors/hive/HiveConnector.h" #include "velox/connectors/hive/HiveDataSource.h" #include "velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.h" // @manual -#include "velox/connectors/hive/storage_adapters/gcs/RegisterGCSFileSystem.h" // @manual +#include "velox/connectors/hive/storage_adapters/gcs/RegisterGcsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/hdfs/RegisterHdfsFileSystem.h" // @manual #include "velox/connectors/hive/storage_adapters/s3fs/RegisterS3FileSystem.h" // @manual #include "velox/dwio/orc/reader/OrcReader.h" @@ -141,10 +142,10 @@ void VeloxBackend::init(const std::unordered_map& conf velox::filesystems::registerS3FileSystem(); #endif #ifdef ENABLE_GCS - velox::filesystems::registerGCSFileSystem(); + velox::filesystems::registerGcsFileSystem(); #endif #ifdef ENABLE_ABFS - velox::filesystems::abfs::registerAbfsFileSystem(); + velox::filesystems::registerAbfsFileSystem(); #endif initJolFilesystem(); @@ -155,6 +156,8 @@ void VeloxBackend::init(const std::unordered_map& conf velox::parquet::registerParquetReaderFactory(); velox::parquet::registerParquetWriterFactory(); velox::orc::registerOrcReaderFactory(); + velox::exec::ExprToSubfieldFilterParser::registerParserFactory( + []() { return std::make_shared(); }); // Register Velox functions registerAllFunctions(); @@ -162,6 +165,10 @@ void VeloxBackend::init(const std::unordered_map& conf // serde, for spill facebook::velox::serializer::presto::PrestoVectorSerde::registerVectorSerde(); } + if (!isRegisteredNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)) { + // RSS shuffle serde. + facebook::velox::serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } velox::exec::Operator::registerOperator(std::make_unique()); initUdf(); @@ -266,8 +273,8 @@ void VeloxBackend::initConnector() { connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedBytes] = backendConf_->get(kMaxCoalescedBytes, "67108864"); // 64M - connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedDistanceBytes] = - backendConf_->get(kMaxCoalescedDistanceBytes, "1048576"); // 1M + connectorConfMap[velox::connector::hive::HiveConfig::kMaxCoalescedDistance] = + backendConf_->get(kMaxCoalescedDistance, "512KB"); // 512KB connectorConfMap[velox::connector::hive::HiveConfig::kPrefetchRowGroups] = backendConf_->get(kPrefetchRowGroups, "1"); connectorConfMap[velox::connector::hive::HiveConfig::kLoadQuantum] = diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc index 332c75dbd725..20c3dec939a0 100644 --- a/cpp/velox/compute/VeloxRuntime.cc +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -28,16 +28,14 @@ #include "compute/VeloxPlanConverter.h" #include "config/VeloxConfig.h" #include "operators/serializer/VeloxRowToColumnarConverter.h" -#include "shuffle/VeloxHashShuffleWriter.h" -#include "shuffle/VeloxRssSortShuffleWriter.h" +#include "operators/writer/VeloxArrowWriter.h" #include "shuffle/VeloxShuffleReader.h" +#include "shuffle/VeloxShuffleWriter.h" #include "utils/ConfigExtractor.h" #include "utils/VeloxArrowUtils.h" #ifdef ENABLE_HDFS - #include "operators/writer/VeloxParquetDataSourceHDFS.h" - #endif #ifdef ENABLE_S3 @@ -259,6 +257,7 @@ std::shared_ptr VeloxRuntime::createShuffleReader( veloxCompressionType, rowType, options.batchSize, + options.bufferSize, memoryManager()->getArrowMemoryPool(), ctxVeloxPool, options.shuffleWriterType); @@ -307,4 +306,12 @@ void VeloxRuntime::dumpConf(const std::string& path) { outFile.close(); } +std::shared_ptr VeloxRuntime::createArrowWriter(const std::string& path) { + int64_t batchSize = 4096; + if (auto it = confMap_.find(kSparkBatchSize); it != confMap_.end()) { + batchSize = std::atol(it->second.c_str()); + } + return std::make_shared(path, batchSize, memoryManager()->getLeafMemoryPool().get()); +} + } // namespace gluten diff --git a/cpp/velox/compute/VeloxRuntime.h b/cpp/velox/compute/VeloxRuntime.h index 846f740cb84c..798fa5bc72f7 100644 --- a/cpp/velox/compute/VeloxRuntime.h +++ b/cpp/velox/compute/VeloxRuntime.h @@ -76,6 +76,8 @@ class VeloxRuntime final : public Runtime { void dumpConf(const std::string& path) override; + std::shared_ptr createArrowWriter(const std::string& path) override; + std::shared_ptr createDataSource(const std::string& filePath, std::shared_ptr schema); std::shared_ptr getVeloxPlan() { diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index 5ece7179b97f..d2f825dcfa8e 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -91,7 +91,7 @@ WholeStageResultIterator::WholeStageResultIterator( std::move(queryCtx), velox::exec::Task::ExecutionMode::kSerial); if (!task_->supportSerialExecutionMode()) { - throw std::runtime_error("Task doesn't support single thread execution: " + planNode->toString()); + throw std::runtime_error("Task doesn't support single threaded execution: " + planNode->toString()); } auto fileSystem = velox::filesystems::getFileSystem(spillDir, nullptr); GLUTEN_CHECK(fileSystem != nullptr, "File System for spilling is null!"); @@ -226,18 +226,6 @@ int64_t WholeStageResultIterator::spillFixedSize(int64_t size) { std::string logPrefix{"Spill[" + poolName + "]: "}; int64_t shrunken = memoryManager_->shrink(size); if (spillStrategy_ == "auto") { - if (task_->numThreads() != 0) { - // Task should have zero running threads, otherwise there's - // possibility that this spill call hangs. See https://github.com/apache/incubator-gluten/issues/7243. - // As of now, non-zero running threads usually happens when: - // 1. Task A spills task B; - // 2. Task A trys to grow buffers created by task B, during which spill is requested on task A again; - LOG(INFO) << fmt::format( - "{} spill is requested on a task {} that has non-zero running threads, which is not currently supported. Skipping.", - logPrefix, - task_->taskId()); - return shrunken; - } int64_t remaining = size - shrunken; LOG(INFO) << fmt::format("{} trying to request spill for {}.", logPrefix, velox::succinctBytes(remaining)); auto mm = memoryManager_->getMemoryManager(); @@ -260,15 +248,47 @@ void WholeStageResultIterator::getOrderedNodeIds( const std::shared_ptr& planNode, std::vector& nodeIds) { bool isProjectNode = (std::dynamic_pointer_cast(planNode) != nullptr); + bool isLocalExchangeNode = (std::dynamic_pointer_cast(planNode) != nullptr); + bool isUnionNode = isLocalExchangeNode && + std::dynamic_pointer_cast(planNode)->type() == + velox::core::LocalPartitionNode::Type::kGather; const auto& sourceNodes = planNode->sources(); - for (const auto& sourceNode : sourceNodes) { + if (isProjectNode) { + GLUTEN_CHECK(sourceNodes.size() == 1, "Illegal state"); + const auto sourceNode = sourceNodes.at(0); // Filter over Project are mapped into FilterProject operator in Velox. // Metrics are all applied on Project node, and the metrics for Filter node // do not exist. - if (isProjectNode && std::dynamic_pointer_cast(sourceNode)) { + if (std::dynamic_pointer_cast(sourceNode)) { omittedNodeIds_.insert(sourceNode->id()); } getOrderedNodeIds(sourceNode, nodeIds); + nodeIds.emplace_back(planNode->id()); + return; + } + + if (isUnionNode) { + // FIXME: The whole metrics system in gluten-substrait is magic. Passing metrics trees through JNI with a trivial + // array is possible but requires for a solid design. Apparently we haven't had it. All the code requires complete + // rework. + // Union was interpreted as LocalPartition + LocalExchange + 2 fake projects as children in Velox. So we only fetch + // metrics from the root node. + std::vector> unionChildren{}; + for (const auto& source : planNode->sources()) { + const auto projectedChild = std::dynamic_pointer_cast(source); + GLUTEN_CHECK(projectedChild != nullptr, "Illegal state"); + const auto projectSources = projectedChild->sources(); + GLUTEN_CHECK(projectSources.size() == 1, "Illegal state"); + const auto projectSource = projectSources.at(0); + getOrderedNodeIds(projectSource, nodeIds); + } + nodeIds.emplace_back(planNode->id()); + return; + } + + for (const auto& sourceNode : sourceNodes) { + // Post-order traversal. + getOrderedNodeIds(sourceNode, nodeIds); } nodeIds.emplace_back(planNode->id()); } @@ -362,9 +382,9 @@ void WholeStageResultIterator::collectMetrics() { continue; } - const auto& status = planStats.at(nodeId); - // Add each operator status into metrics. - for (const auto& entry : status.operatorStats) { + const auto& stats = planStats.at(nodeId); + // Add each operator stats into metrics. + for (const auto& entry : stats.operatorStats) { const auto& second = entry.second; metrics_->get(Metrics::kInputRows)[metricIndex] = second->inputRows; metrics_->get(Metrics::kInputVectors)[metricIndex] = second->inputVectors; @@ -442,8 +462,6 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kSparkBatchSize, 4096)); configs[velox::core::QueryConfig::kMaxOutputBatchRows] = std::to_string(veloxCfg_->get(kSparkBatchSize, 4096)); - // Find offheap size from Spark confs. If found, set the max memory usage of partial aggregation. - // FIXME this uses process-wise off-heap memory which is not for task try { if (veloxCfg_->valueExists(kDefaultSessionTimezone)) { configs[velox::core::QueryConfig::kSessionTimezone] = veloxCfg_->get(kDefaultSessionTimezone, ""); @@ -455,12 +473,14 @@ std::unordered_map WholeStageResultIterator::getQueryC configs[velox::core::QueryConfig::kAdjustTimestampToTimezone] = "true"; { + // Find offheap size from Spark confs. If found, set the max memory usage of partial aggregation. + // FIXME this uses process-wise off-heap memory which is not for task // partial aggregation memory config auto offHeapMemory = veloxCfg_->get(kSparkTaskOffHeapMemory, facebook::velox::memory::kMaxMemory); auto maxPartialAggregationMemory = - (long)(veloxCfg_->get(kMaxPartialAggregationMemoryRatio, 0.1) * offHeapMemory); + static_cast((veloxCfg_->get(kMaxPartialAggregationMemoryRatio, 0.1) * offHeapMemory)); auto maxExtendedPartialAggregationMemory = - (long)(veloxCfg_->get(kMaxExtendedPartialAggregationMemoryRatio, 0.15) * offHeapMemory); + static_cast((veloxCfg_->get(kMaxExtendedPartialAggregationMemoryRatio, 0.15) * offHeapMemory)); configs[velox::core::QueryConfig::kMaxPartialAggregationMemory] = std::to_string(maxPartialAggregationMemory); configs[velox::core::QueryConfig::kMaxExtendedPartialAggregationMemory] = std::to_string(maxExtendedPartialAggregationMemory); @@ -468,8 +488,6 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kAbandonPartialAggregationMinPct, 90)); configs[velox::core::QueryConfig::kAbandonPartialAggregationMinRows] = std::to_string(veloxCfg_->get(kAbandonPartialAggregationMinRows, 100000)); - // Spark's collect_set ignore nulls. - configs[velox::core::QueryConfig::kPrestoArrayAggIgnoreNulls] = "true"; } // Spill configs if (spillStrategy_ == "none") { @@ -494,12 +512,14 @@ std::unordered_map WholeStageResultIterator::getQueryC std::to_string(veloxCfg_->get(kSpillWriteBufferSize, 4L * 1024 * 1024)); configs[velox::core::QueryConfig::kSpillStartPartitionBit] = std::to_string(veloxCfg_->get(kSpillStartPartitionBit, 29)); - configs[velox::core::QueryConfig::kJoinSpillPartitionBits] = - std::to_string(veloxCfg_->get(kSpillPartitionBits, 2)); + configs[velox::core::QueryConfig::kSpillNumPartitionBits] = + std::to_string(veloxCfg_->get(kSpillPartitionBits, 3)); configs[velox::core::QueryConfig::kSpillableReservationGrowthPct] = std::to_string(veloxCfg_->get(kSpillableReservationGrowthPct, 25)); configs[velox::core::QueryConfig::kSpillCompressionKind] = veloxCfg_->get(kSpillCompressionKind, "lz4"); + configs[velox::core::QueryConfig::kSpillPrefixSortEnabled] = + veloxCfg_->get(kSpillPrefixSortEnabled, "false"); configs[velox::core::QueryConfig::kSparkBloomFilterExpectedNumItems] = std::to_string(veloxCfg_->get(kBloomFilterExpectedNumItems, 1000000)); configs[velox::core::QueryConfig::kSparkBloomFilterNumBits] = diff --git a/cpp/velox/compute/WholeStageResultIterator.h b/cpp/velox/compute/WholeStageResultIterator.h index 9b1f2b728802..d0dd47bfd1e3 100644 --- a/cpp/velox/compute/WholeStageResultIterator.h +++ b/cpp/velox/compute/WholeStageResultIterator.h @@ -120,7 +120,7 @@ class WholeStageResultIterator : public ColumnarBatchIterator { /// All the children plan node ids with postorder traversal. std::vector orderedNodeIds_; - /// Node ids should be ommited in metrics. + /// Node ids should be omitted in metrics. std::unordered_set omittedNodeIds_; std::vector scanNodeIds_; std::vector> scanInfos_; diff --git a/cpp/velox/config/VeloxConfig.h b/cpp/velox/config/VeloxConfig.h index 792beda96f7d..4ae82f263d37 100644 --- a/cpp/velox/config/VeloxConfig.h +++ b/cpp/velox/config/VeloxConfig.h @@ -44,6 +44,7 @@ const uint64_t kMaxSpillFileSizeDefault = 1L * 1024 * 1024 * 1024; const std::string kSpillableReservationGrowthPct = "spark.gluten.sql.columnar.backend.velox.spillableReservationGrowthPct"; const std::string kSpillCompressionKind = "spark.io.compression.codec"; +const std::string kSpillPrefixSortEnabled = "spark.gluten.sql.columnar.backend.velox.spillPrefixsortEnabled"; const std::string kMaxPartialAggregationMemoryRatio = "spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio"; const std::string kMaxExtendedPartialAggregationMemoryRatio = @@ -76,6 +77,9 @@ const bool kMemoryUseHugePagesDefault = false; const std::string kVeloxMemInitCapacity = "spark.gluten.sql.columnar.backend.velox.memInitCapacity"; const uint64_t kVeloxMemInitCapacityDefault = 8 << 20; +const std::string kVeloxMemReclaimMaxWaitMs = "spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs"; +const uint64_t kVeloxMemReclaimMaxWaitMsDefault = 3600000; // 60min + const std::string kHiveConnectorId = "test-hive"; const std::string kVeloxCacheEnabled = "spark.gluten.sql.columnar.backend.velox.cacheEnabled"; @@ -115,7 +119,7 @@ const std::string kDirectorySizeGuess = "spark.gluten.sql.columnar.backend.velox const std::string kFilePreloadThreshold = "spark.gluten.sql.columnar.backend.velox.filePreloadThreshold"; const std::string kPrefetchRowGroups = "spark.gluten.sql.columnar.backend.velox.prefetchRowGroups"; const std::string kLoadQuantum = "spark.gluten.sql.columnar.backend.velox.loadQuantum"; -const std::string kMaxCoalescedDistanceBytes = "spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes"; +const std::string kMaxCoalescedDistance = "spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance"; const std::string kMaxCoalescedBytes = "spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes"; const std::string kCachePrefetchMinPct = "spark.gluten.sql.columnar.backend.velox.cachePrefetchMinPct"; diff --git a/cpp/velox/jni/JniFileSystem.cc b/cpp/velox/jni/JniFileSystem.cc index 21cdf2b23a02..fa849664e3ae 100644 --- a/cpp/velox/jni/JniFileSystem.cc +++ b/cpp/velox/jni/JniFileSystem.cc @@ -241,7 +241,7 @@ class FileSystemWrapper : public facebook::velox::filesystems::FileSystem { return fs_->list(rewrite(path)); } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { fs_->mkdir(rewrite(path)); } @@ -349,7 +349,7 @@ class JniFileSystem : public facebook::velox::filesystems::FileSystem { return out; } - void mkdir(std::string_view path) override { + void mkdir(std::string_view path, const facebook::velox::filesystems::DirectoryOptions& options = {}) override { JNIEnv* env = nullptr; attachCurrentThreadAsDaemonOrThrow(vm, &env); env->CallVoidMethod(obj_, jniFileSystemMkdir, createJString(env, path)); diff --git a/cpp/velox/jni/JniFileSystem.h b/cpp/velox/jni/JniFileSystem.h index 9313e7af7b09..11c0f27b369e 100644 --- a/cpp/velox/jni/JniFileSystem.h +++ b/cpp/velox/jni/JniFileSystem.h @@ -24,9 +24,9 @@ namespace gluten { -// Register JNI-or-local (or JVM-over-local, as long as it describes what happens here) -// file system. maxFileSize is necessary (!= 0) because we use this size to decide -// whether a new file can fit in JVM heap, otherwise we write it via local fs directly. +// Register JNI-or-local (or JVM-over-local, as long as it describes what happens here) file system. maxFileSize is +// necessary (!= 0) because we use this size to decide whether a new file can fit in JVM heap, otherwise we write it via +// local fs directly. void registerJolFileSystem(uint64_t maxFileSize); void initVeloxJniFileSystem(JNIEnv* env); diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index b8d2b0c3c2ff..9d6ad157ff3a 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -41,13 +41,15 @@ using namespace gluten; using namespace facebook; +namespace { +jclass blockStripesClass; +jmethodID blockStripesConstructor; +} // namespace + #ifdef __cplusplus extern "C" { #endif -static jclass blockStripesClass; -static jmethodID blockStripesConstructor; - jint JNI_OnLoad(JavaVM* vm, void*) { JNIEnv* env; if (vm->GetEnv(reinterpret_cast(&env), jniVersion) != JNI_OK) { @@ -385,7 +387,6 @@ JNIEXPORT void JNICALL Java_org_apache_gluten_datasource_VeloxDataSourceJniWrapp jlong dsHandle, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getRuntime(env, wrapper); auto datasource = ObjectStore::retrieve(dsHandle); auto batch = ObjectStore::retrieve(batchHandle); datasource->write(batch); diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index 19a5d458049b..5e6fdd08a2db 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -41,6 +41,8 @@ static constexpr std::string_view kMemoryPoolInitialCapacity{"memory-pool-initia static constexpr uint64_t kDefaultMemoryPoolInitialCapacity{256 << 20}; static constexpr std::string_view kMemoryPoolTransferCapacity{"memory-pool-transfer-capacity"}; static constexpr uint64_t kDefaultMemoryPoolTransferCapacity{128 << 20}; +static constexpr std::string_view kMemoryReclaimMaxWaitMs{"memory-reclaim-max-wait-time"}; +static constexpr std::string_view kDefaultMemoryReclaimMaxWaitMs{"3600000ms"}; template T getConfig( @@ -57,6 +59,7 @@ T getConfig( return defaultValue; } } // namespace + /// We assume in a single Spark task. No thread-safety should be guaranteed. class ListenableArbitrator : public velox::memory::MemoryArbitrator { public: @@ -74,7 +77,13 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { config.extraConfigs, kMemoryPoolTransferCapacity, std::to_string(kDefaultMemoryPoolTransferCapacity)), - velox::config::CapacityUnit::BYTE)) {} + velox::config::CapacityUnit::BYTE)), + memoryReclaimMaxWaitMs_( + std::chrono::duration_cast(velox::config::toDuration(getConfig( + config.extraConfigs, + kMemoryReclaimMaxWaitMs, + std::string(kDefaultMemoryReclaimMaxWaitMs)))) + .count()) {} std::string kind() const override { return kind_; } @@ -99,7 +108,9 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } bool growCapacity(velox::memory::MemoryPool* pool, uint64_t targetBytes) override { - velox::memory::ScopedMemoryArbitrationContext ctx(pool); + // Set arbitration context to allow memory over-use during recursive arbitration. + // See MemoryPoolImpl::maybeIncrementReservation. + velox::memory::ScopedMemoryArbitrationContext ctx{}; velox::memory::MemoryPool* candidate; { std::unique_lock guard{mutex_}; @@ -121,7 +132,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { VELOX_CHECK_EQ(candidates_.size(), 1, "ListenableArbitrator should only be used within a single root pool"); pool = candidates_.begin()->first; } - pool->reclaim(targetBytes, 0, status); // ignore the output + pool->reclaim(targetBytes, memoryReclaimMaxWaitMs_, status); // ignore the output return shrinkCapacity0(pool, 0); } @@ -170,6 +181,7 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { gluten::AllocationListener* listener_; const uint64_t memoryPoolInitialCapacity_; // FIXME: Unused. const uint64_t memoryPoolTransferCapacity_; + const uint64_t memoryReclaimMaxWaitMs_; mutable std::mutex mutex_; inline static std::string kind_ = "GLUTEN"; @@ -208,14 +220,16 @@ VeloxMemoryManager::VeloxMemoryManager(const std::string& kind, std::unique_ptr< kMemoryReservationBlockSize, kMemoryReservationBlockSizeDefault); auto memInitCapacity = VeloxBackend::get()->getBackendConf()->get(kVeloxMemInitCapacity, kVeloxMemInitCapacityDefault); + auto memReclaimMaxWaitMs = + VeloxBackend::get()->getBackendConf()->get(kVeloxMemReclaimMaxWaitMs, kVeloxMemReclaimMaxWaitMsDefault); blockListener_ = std::make_unique(listener_.get(), reservationBlockSize); listenableAlloc_ = std::make_unique(defaultMemoryAllocator().get(), blockListener_.get()); arrowPool_ = std::make_unique(listenableAlloc_.get()); std::unordered_map extraArbitratorConfigs; - extraArbitratorConfigs["memory-pool-initial-capacity"] = folly::to(memInitCapacity) + "B"; - extraArbitratorConfigs["memory-pool-transfer-capacity"] = folly::to(reservationBlockSize) + "B"; - extraArbitratorConfigs["memory-reclaim-max-wait-time"] = folly::to(0) + "ms"; + extraArbitratorConfigs[std::string(kMemoryPoolInitialCapacity)] = folly::to(memInitCapacity) + "B"; + extraArbitratorConfigs[std::string(kMemoryPoolTransferCapacity)] = folly::to(reservationBlockSize) + "B"; + extraArbitratorConfigs[std::string(kMemoryReclaimMaxWaitMs)] = folly::to(memReclaimMaxWaitMs) + "ms"; ArbitratorFactoryRegister afr(listener_.get()); velox::memory::MemoryManagerOptions mmOptions{ @@ -380,7 +394,7 @@ VeloxMemoryManager::~VeloxMemoryManager() { << "ms as there are still outstanding memory resources. "; } #ifdef ENABLE_JEMALLOC_STATS - je_gluten_malloc_stats_print(NULL, NULL, NULL); + malloc_stats_print(NULL, NULL, NULL); #endif } diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index 6e2f90f0105b..5d46dbdcdd0e 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -29,8 +29,8 @@ #include "velox/functions/sparksql/DecimalArithmetic.h" #include "velox/functions/sparksql/Hash.h" #include "velox/functions/sparksql/Rand.h" -#include "velox/functions/sparksql/Register.h" #include "velox/functions/sparksql/aggregates/Register.h" +#include "velox/functions/sparksql/registration/Register.h" #include "velox/functions/sparksql/window/WindowFunctionsRegistration.h" using namespace facebook; @@ -43,8 +43,10 @@ void registerPrestoVectorFunctions() { VELOX_REGISTER_VECTOR_FUNCTION(udf_transform_values, "transform_values"); } } // namespace facebook::velox::functions + namespace gluten { namespace { + void registerFunctionOverwrite() { velox::functions::registerUnaryNumeric({"round"}); velox::registerFunction({"round"}); @@ -76,13 +78,6 @@ void registerFunctionOverwrite() { velox::functions::registerPrestoVectorFunctions(); } -void registerFunctionForConfig() { - const std::string prefix = "not_allow_precision_loss_"; - velox::functions::sparksql::registerDecimalAdd(prefix, false); - velox::functions::sparksql::registerDecimalSubtract(prefix, false); - velox::functions::sparksql::registerDecimalMultiply(prefix, false); - velox::functions::sparksql::registerDecimalDivide(prefix, false); -} } // namespace void registerAllFunctions() { @@ -96,7 +91,6 @@ void registerAllFunctions() { // Using function overwrite to handle function names mismatch between Spark // and Velox. registerFunctionOverwrite(); - registerFunctionForConfig(); } } // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.cc b/cpp/velox/operators/functions/RowConstructorWithNull.cc index e8b8a288360b..4d052e0df279 100644 --- a/cpp/velox/operators/functions/RowConstructorWithNull.cc +++ b/cpp/velox/operators/functions/RowConstructorWithNull.cc @@ -19,6 +19,7 @@ #include "velox/expression/VectorFunction.h" namespace gluten { + facebook::velox::TypePtr RowConstructorWithNullCallToSpecialForm::resolveType( const std::vector& argTypes) { auto numInput = argTypes.size(); @@ -52,4 +53,5 @@ facebook::velox::exec::ExprPtr RowConstructorWithNullCallToSpecialForm::construc return std::make_shared( type, std::move(compiledChildren), function, metadata, name, trackCpuUsage); } + } // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.h b/cpp/velox/operators/functions/RowConstructorWithNull.h index 66b745e3ed9b..2565ba42f382 100644 --- a/cpp/velox/operators/functions/RowConstructorWithNull.h +++ b/cpp/velox/operators/functions/RowConstructorWithNull.h @@ -21,6 +21,7 @@ #include "velox/expression/SpecialForm.h" namespace gluten { + class RowConstructorWithNullCallToSpecialForm : public facebook::velox::exec::FunctionCallToSpecialForm { public: RowConstructorWithNullCallToSpecialForm(const std::string& rowFunctionName) { @@ -49,4 +50,5 @@ class RowConstructorWithNullCallToSpecialForm : public facebook::velox::exec::Fu private: std::string rowFunctionName; }; + } // namespace gluten diff --git a/cpp/velox/operators/functions/RowFunctionWithNull.h b/cpp/velox/operators/functions/RowFunctionWithNull.h index 4131fb472ddd..6477449fb0dc 100644 --- a/cpp/velox/operators/functions/RowFunctionWithNull.h +++ b/cpp/velox/operators/functions/RowFunctionWithNull.h @@ -22,10 +22,8 @@ namespace gluten { -/** - * @tparam allNull If true, set struct as null when all of arguments are all, else will - * set it null when one of its arguments is null. - */ +///@tparam allNull If true, set struct as null when all of arguments are all, else will +/// set it null when one of its arguments is null. template class RowFunctionWithNull final : public facebook::velox::exec::VectorFunction { public: diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc new file mode 100644 index 000000000000..26614690e141 --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.cc @@ -0,0 +1,104 @@ +/* + * 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. + */ +#include "operators/functions/SparkExprToSubfieldFilterParser.h" + +namespace gluten { + +using namespace facebook::velox; + +bool SparkExprToSubfieldFilterParser::toSparkSubfield(const core::ITypedExpr* field, common::Subfield& subfield) { + std::vector> path; + for (auto* current = field;;) { + if (auto* fieldAccess = dynamic_cast(current)) { + path.push_back(std::make_unique(fieldAccess->name())); + } else if (dynamic_cast(current)) { + return false; + } else if (dynamic_cast(current) == nullptr) { + return false; + } else { + break; + } + + if (!current->inputs().empty()) { + return false; + } else { + break; + } + } + std::reverse(path.begin(), path.end()); + subfield = common::Subfield(std::move(path)); + return true; +} + +std::unique_ptr SparkExprToSubfieldFilterParser::leafCallToSubfieldFilter( + const core::CallTypedExpr& call, + common::Subfield& subfield, + core::ExpressionEvaluator* evaluator, + bool negated) { + if (call.inputs().empty()) { + return nullptr; + } + + const auto* leftSide = call.inputs()[0].get(); + + if (call.name() == "equalto") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeNotEqualFilter(call.inputs()[1], evaluator) : makeEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanFilter(call.inputs()[1], evaluator) + : makeLessThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "lessthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator) + : makeLessThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthanorequal") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanFilter(call.inputs()[1], evaluator) + : makeGreaterThanOrEqualFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "greaterthan") { + if (toSparkSubfield(leftSide, subfield)) { + return negated ? makeLessThanOrEqualFilter(call.inputs()[1], evaluator) + : makeGreaterThanFilter(call.inputs()[1], evaluator); + } + } else if (call.name() == "in") { + if (toSparkSubfield(leftSide, subfield)) { + return makeInFilter(call.inputs()[1], evaluator, negated); + } + } else if (call.name() == "isnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNotNull(); + } + return exec::isNull(); + } + } else if (call.name() == "isnotnull") { + if (toSparkSubfield(leftSide, subfield)) { + if (negated) { + return exec::isNull(); + } + return exec::isNotNull(); + } + } + return nullptr; +} + +} // namespace gluten diff --git a/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h new file mode 100644 index 000000000000..11e1b401a299 --- /dev/null +++ b/cpp/velox/operators/functions/SparkExprToSubfieldFilterParser.h @@ -0,0 +1,38 @@ +/* + * 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. + */ +#include "velox/expression/ExprToSubfieldFilter.h" + +namespace gluten { + +/// Parses Spark expression into subfield filter. Differences from Presto's parser include: +/// 1) Some Spark functions are registered under different names. +/// 2) The supported functions vary. +/// 3) Filter push-down on nested fields is disabled. +class SparkExprToSubfieldFilterParser : public facebook::velox::exec::ExprToSubfieldFilterParser { + public: + std::unique_ptr leafCallToSubfieldFilter( + const facebook::velox::core::CallTypedExpr& call, + facebook::velox::common::Subfield& subfield, + facebook::velox::core::ExpressionEvaluator* evaluator, + bool negated) override; + + private: + // Compared to the upstream 'toSubfield', the push-down of filter on nested field is disabled. + bool toSparkSubfield(const facebook::velox::core::ITypedExpr* field, facebook::velox::common::Subfield& subfield); +}; + +} // namespace gluten diff --git a/cpp/velox/operators/plannodes/RowVectorStream.h b/cpp/velox/operators/plannodes/RowVectorStream.h index c72e9137f4a4..d503eb2d14c3 100644 --- a/cpp/velox/operators/plannodes/RowVectorStream.h +++ b/cpp/velox/operators/plannodes/RowVectorStream.h @@ -21,29 +21,78 @@ #include "memory/VeloxColumnarBatch.h" #include "velox/exec/Driver.h" #include "velox/exec/Operator.h" +#include "velox/exec/Task.h" + +namespace { + +class SuspendedSection { + public: + explicit SuspendedSection(facebook::velox::exec::Driver* driver) : driver_(driver) { + if (driver_->task()->enterSuspended(driver->state()) != facebook::velox::exec::StopReason::kNone) { + VELOX_FAIL("Terminate detected when entering suspended section"); + } + } + + virtual ~SuspendedSection() { + if (driver_->task()->leaveSuspended(driver_->state()) != facebook::velox::exec::StopReason::kNone) { + LOG(WARNING) << "Terminate detected when leaving suspended section for driver " << driver_->driverCtx()->driverId + << " from task " << driver_->task()->taskId(); + } + } + + private: + facebook::velox::exec::Driver* const driver_; +}; + +} // namespace namespace gluten { + class RowVectorStream { public: explicit RowVectorStream( + facebook::velox::exec::DriverCtx* driverCtx, facebook::velox::memory::MemoryPool* pool, - std::shared_ptr iterator, + ResultIterator* iterator, const facebook::velox::RowTypePtr& outputType) - : iterator_(std::move(iterator)), outputType_(outputType), pool_(pool) {} + : driverCtx_(driverCtx), pool_(pool), outputType_(outputType), iterator_(iterator) {} bool hasNext() { - if (!finished_) { - finished_ = !iterator_->hasNext(); + if (finished_) { + return false; + } + bool hasNext; + { + // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current + // driver to make the current task open to spilling. + // + // When a task is getting spilled, it should have been suspended so has zero running threads, otherwise there's + // possibility that this spill call hangs. See https://github.com/apache/incubator-gluten/issues/7243. + // As of now, non-zero running threads usually happens when: + // 1. Task A spills task B; + // 2. Task A tries to grow buffers created by task B, during which spill is requested on task A again. + SuspendedSection ss(driverCtx_->driver); + hasNext = iterator_->hasNext(); + } + if (!hasNext) { + finished_ = true; } - return !finished_; + return hasNext; } - // Convert arrow batch to rowvector and use new output columns + // Convert arrow batch to row vector and use new output columns facebook::velox::RowVectorPtr next() { if (finished_) { return nullptr; } - const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, iterator_->next()); + std::shared_ptr cb; + { + // We are leaving Velox task execution and are probably entering Spark code through JNI. Suspend the current + // driver to make the current task open to spilling. + SuspendedSection ss(driverCtx_->driver); + cb = iterator_->next(); + } + const std::shared_ptr& vb = VeloxColumnarBatch::from(pool_, cb); auto vp = vb->getRowVector(); VELOX_DCHECK(vp != nullptr); return std::make_shared( @@ -51,10 +100,12 @@ class RowVectorStream { } private: - bool finished_{false}; - std::shared_ptr iterator_; - const facebook::velox::RowTypePtr outputType_; + facebook::velox::exec::DriverCtx* driverCtx_; facebook::velox::memory::MemoryPool* pool_; + const facebook::velox::RowTypePtr outputType_; + ResultIterator* iterator_; + + bool finished_{false}; }; class ValueStreamNode final : public facebook::velox::core::PlanNode { @@ -62,21 +113,19 @@ class ValueStreamNode final : public facebook::velox::core::PlanNode { ValueStreamNode( const facebook::velox::core::PlanNodeId& id, const facebook::velox::RowTypePtr& outputType, - std::unique_ptr valueStream) - : facebook::velox::core::PlanNode(id), outputType_(outputType), valueStream_(std::move(valueStream)) { - VELOX_CHECK_NOT_NULL(valueStream_); - } + std::shared_ptr iterator) + : facebook::velox::core::PlanNode(id), outputType_(outputType), iterator_(std::move(iterator)) {} const facebook::velox::RowTypePtr& outputType() const override { return outputType_; } const std::vector& sources() const override { - return kEmptySources; + return kEmptySources_; }; - RowVectorStream* rowVectorStream() const { - return valueStream_.get(); + ResultIterator* iterator() const { + return iterator_.get(); } std::string_view name() const override { @@ -91,8 +140,8 @@ class ValueStreamNode final : public facebook::velox::core::PlanNode { void addDetails(std::stringstream& stream) const override{}; const facebook::velox::RowTypePtr outputType_; - std::unique_ptr valueStream_; - const std::vector kEmptySources; + std::shared_ptr iterator_; + const std::vector kEmptySources_; }; class ValueStream : public facebook::velox::exec::SourceOperator { @@ -107,15 +156,17 @@ class ValueStream : public facebook::velox::exec::SourceOperator { operatorId, valueStreamNode->id(), valueStreamNode->name().data()) { - valueStream_ = valueStreamNode->rowVectorStream(); + ResultIterator* itr = valueStreamNode->iterator(); + VELOX_CHECK_NOT_NULL(itr); + rvStream_ = std::make_unique(driverCtx, pool(), itr, outputType_); } facebook::velox::RowVectorPtr getOutput() override { if (finished_) { return nullptr; } - if (valueStream_->hasNext()) { - return valueStream_->next(); + if (rvStream_->hasNext()) { + return rvStream_->next(); } else { finished_ = true; return nullptr; @@ -132,7 +183,7 @@ class ValueStream : public facebook::velox::exec::SourceOperator { private: bool finished_ = false; - RowVectorStream* valueStream_; + std::unique_ptr rvStream_; }; class RowVectorStreamOperatorTranslator : public facebook::velox::exec::Operator::PlanNodeTranslator { diff --git a/cpp/velox/benchmarks/common/FileReaderIterator.cc b/cpp/velox/operators/reader/FileReaderIterator.cc similarity index 55% rename from cpp/velox/benchmarks/common/FileReaderIterator.cc rename to cpp/velox/operators/reader/FileReaderIterator.cc index 26985c7f03c4..49d52f5ffd06 100644 --- a/cpp/velox/benchmarks/common/FileReaderIterator.cc +++ b/cpp/velox/operators/reader/FileReaderIterator.cc @@ -15,33 +15,39 @@ * limitations under the License. */ -#include "FileReaderIterator.h" -#include "benchmarks/common/ParquetReaderIterator.h" -#ifdef GLUTEN_ENABLE_ORC -#include "benchmarks/common/OrcReaderIterator.h" -#endif +#include "operators/reader/FileReaderIterator.h" +#include +#include "operators/reader/ParquetReaderIterator.h" -std::shared_ptr gluten::getInputIteratorFromFileReader( +namespace gluten { +namespace { +const std::string kParquetSuffix = ".parquet"; +} + +FileReaderIterator::FileReaderIterator(const std::string& path) : path_(path) {} + +int64_t FileReaderIterator::getCollectBatchTime() const { + return collectBatchTime_; +} + +std::shared_ptr FileReaderIterator::getInputIteratorFromFileReader( + FileReaderType readerType, const std::string& path, - gluten::FileReaderType readerType) { + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) { std::filesystem::path input{path}; auto suffix = input.extension().string(); if (suffix == kParquetSuffix) { if (readerType == FileReaderType::kStream) { - return std::make_shared(std::make_unique(path)); + return std::make_shared( + std::make_unique(path, batchSize, pool)); } if (readerType == FileReaderType::kBuffered) { - return std::make_shared(std::make_unique(path)); + return std::make_shared( + std::make_unique(path, batchSize, pool)); } - } else if (suffix == kOrcSuffix) { -#ifdef GLUTEN_ENABLE_ORC - if (readerType == FileReaderType::kStream) { - return std::make_shared(std::make_unique(path)); - } - if (readerType == FileReaderType::kBuffered) { - return std::make_shared(std::make_unique(path)); - } -#endif } throw new GlutenException("Unreachable."); } + +} // namespace gluten diff --git a/cpp/velox/benchmarks/common/FileReaderIterator.h b/cpp/velox/operators/reader/FileReaderIterator.h similarity index 69% rename from cpp/velox/benchmarks/common/FileReaderIterator.h rename to cpp/velox/operators/reader/FileReaderIterator.h index 16db58ce4569..708120603e42 100644 --- a/cpp/velox/benchmarks/common/FileReaderIterator.h +++ b/cpp/velox/operators/reader/FileReaderIterator.h @@ -14,43 +14,36 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#pragma once -#include -#include -#include +#pragma once -#include "BenchmarkUtils.h" #include "compute/ResultIterator.h" -#include "memory/ColumnarBatch.h" #include "memory/ColumnarBatchIterator.h" +#include "velox/common/memory/MemoryPool.h" namespace gluten { -static const std::string kOrcSuffix = ".orc"; -static const std::string kParquetSuffix = ".parquet"; - enum FileReaderType { kBuffered, kStream, kNone }; class FileReaderIterator : public ColumnarBatchIterator { public: - explicit FileReaderIterator(const std::string& path) : path_(path) {} + static std::shared_ptr getInputIteratorFromFileReader( + FileReaderType readerType, + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool); + + explicit FileReaderIterator(const std::string& path); virtual ~FileReaderIterator() = default; virtual std::shared_ptr getSchema() = 0; - int64_t getCollectBatchTime() const { - return collectBatchTime_; - } + int64_t getCollectBatchTime() const; protected: int64_t collectBatchTime_ = 0; std::string path_; }; -std::shared_ptr getInputIteratorFromFileReader( - const std::string& path, - FileReaderType readerType); - } // namespace gluten diff --git a/cpp/velox/operators/reader/ParquetReaderIterator.cc b/cpp/velox/operators/reader/ParquetReaderIterator.cc new file mode 100644 index 000000000000..014bccbfce9c --- /dev/null +++ b/cpp/velox/operators/reader/ParquetReaderIterator.cc @@ -0,0 +1,98 @@ +/* + * 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. + */ + +#include "operators/reader/ParquetReaderIterator.h" +#include "memory/VeloxColumnarBatch.h" + +#include + +namespace gluten { + +ParquetReaderIterator::ParquetReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : FileReaderIterator(path), batchSize_(batchSize), pool_(pool) {} + +void ParquetReaderIterator::createReader() { + parquet::ArrowReaderProperties properties = parquet::default_arrow_reader_properties(); + properties.set_batch_size(batchSize_); + GLUTEN_THROW_NOT_OK(parquet::arrow::FileReader::Make( + arrow::default_memory_pool(), parquet::ParquetFileReader::OpenFile(path_), properties, &fileReader_)); + GLUTEN_THROW_NOT_OK( + fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); + + auto schema = recordBatchReader_->schema(); + DLOG(INFO) << "Schema:\n" << schema->ToString(); +} + +std::shared_ptr ParquetReaderIterator::getSchema() { + return recordBatchReader_->schema(); +} + +ParquetStreamReaderIterator::ParquetStreamReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ParquetReaderIterator(path, batchSize, pool) { + createReader(); + DLOG(INFO) << "ParquetStreamReaderIterator open file: " << path; +} + +std::shared_ptr ParquetStreamReaderIterator::next() { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); + DLOG(INFO) << "ParquetStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0); + collectBatchTime_ += + std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); + if (batch == nullptr) { + return nullptr; + } + return VeloxColumnarBatch::from(pool_, std::make_shared(batch)); +} + +ParquetBufferedReaderIterator::ParquetBufferedReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ParquetReaderIterator(path, batchSize, pool) { + createReader(); + collectBatches(); + iter_ = batches_.begin(); + DLOG(INFO) << "ParquetBufferedReaderIterator open file: " << path; + DLOG(INFO) << "Number of input batches: " << std::to_string(batches_.size()); + if (iter_ != batches_.cend()) { + DLOG(INFO) << "columns: " << (*iter_)->num_columns(); + DLOG(INFO) << "rows: " << (*iter_)->num_rows(); + } +} + +std::shared_ptr ParquetBufferedReaderIterator::next() { + if (iter_ == batches_.cend()) { + return nullptr; + } + return VeloxColumnarBatch::from(pool_, std::make_shared(*iter_++)); +} + +void ParquetBufferedReaderIterator::collectBatches() { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(batches_, recordBatchReader_->ToRecordBatches()); + auto endTime = std::chrono::steady_clock::now(); + collectBatchTime_ += std::chrono::duration_cast(endTime - startTime).count(); +} + +} // namespace gluten diff --git a/cpp/velox/operators/reader/ParquetReaderIterator.h b/cpp/velox/operators/reader/ParquetReaderIterator.h new file mode 100644 index 000000000000..f45fe5eb7721 --- /dev/null +++ b/cpp/velox/operators/reader/ParquetReaderIterator.h @@ -0,0 +1,65 @@ +/* + * 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. + */ + +#pragma once + +#include "operators/reader/FileReaderIterator.h" + +#include +#include + +namespace gluten { + +class ParquetReaderIterator : public FileReaderIterator { + public: + explicit ParquetReaderIterator(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + void createReader(); + + std::shared_ptr getSchema() override; + + protected: + std::unique_ptr<::parquet::arrow::FileReader> fileReader_; + std::shared_ptr recordBatchReader_; + int64_t batchSize_; + facebook::velox::memory::MemoryPool* pool_; +}; + +class ParquetStreamReaderIterator final : public ParquetReaderIterator { + public: + ParquetStreamReaderIterator(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr next() override; +}; + +class ParquetBufferedReaderIterator final : public ParquetReaderIterator { + public: + explicit ParquetBufferedReaderIterator( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr next() override; + + private: + void collectBatches(); + + arrow::RecordBatchVector batches_; + std::vector>::const_iterator iter_; +}; + +} // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index acb14cf4de39..7b858cce21b2 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -17,10 +17,11 @@ #include "VeloxColumnarBatchSerializer.h" +#include + #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "velox/common/memory/Memory.h" -#include "velox/vector/ComplexVector.h" #include "velox/vector/FlatVector.h" #include "velox/vector/arrow/Bridge.h" @@ -29,14 +30,15 @@ using namespace facebook::velox; namespace gluten { - namespace { + std::unique_ptr toByteStream(uint8_t* data, int32_t size) { std::vector byteRanges; byteRanges.push_back(ByteRange{data, size, 0}); auto byteStream = std::make_unique(byteRanges); return byteStream; } + } // namespace VeloxColumnarBatchSerializer::VeloxColumnarBatchSerializer( @@ -88,4 +90,5 @@ std::shared_ptr VeloxColumnarBatchSerializer::deserialize(uint8_t serde_->deserialize(byteStream.get(), veloxPool_.get(), rowType_, &result, &options_); return std::make_shared(result); } + } // namespace gluten diff --git a/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc b/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc index 5dec7db23af4..19a2bbafd2c2 100644 --- a/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc +++ b/cpp/velox/operators/serializer/VeloxRowToColumnarConverter.cc @@ -255,6 +255,7 @@ bool supporteType(const RowTypePtr rowType) { } } // namespace + VeloxRowToColumnarConverter::VeloxRowToColumnarConverter( struct ArrowSchema* cSchema, std::shared_ptr memoryPool) @@ -301,4 +302,5 @@ VeloxRowToColumnarConverter::convertPrimitive(int64_t numRows, int64_t* rowLengt auto rowVector = std::make_shared(pool_.get(), rowType_, BufferPtr(nullptr), numRows, std::move(columns)); return std::make_shared(rowVector); } + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.cc b/cpp/velox/operators/writer/VeloxArrowWriter.cc new file mode 100644 index 000000000000..e01c0aee9fe2 --- /dev/null +++ b/cpp/velox/operators/writer/VeloxArrowWriter.cc @@ -0,0 +1,39 @@ +/* + * 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. + */ + +#include "operators/writer/VeloxArrowWriter.h" + +namespace gluten { + +VeloxArrowWriter::VeloxArrowWriter( + const std::string& path, + int64_t batchSize, + facebook::velox::memory::MemoryPool* pool) + : ArrowWriter(path), batchSize_(batchSize), pool_(pool) {} + +std::shared_ptr VeloxArrowWriter::retrieveColumnarBatch() { + if (writer_ == nullptr) { + // No data to read. + return nullptr; + } + if (reader_ == nullptr) { + reader_ = std::make_unique(path_, batchSize_, pool_); + } + return reader_->next(); +} + +} // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxArrowWriter.h b/cpp/velox/operators/writer/VeloxArrowWriter.h new file mode 100644 index 000000000000..e0a0861b3bc2 --- /dev/null +++ b/cpp/velox/operators/writer/VeloxArrowWriter.h @@ -0,0 +1,37 @@ +/* + * 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. + */ + +#pragma once + +#include "operators/reader/ParquetReaderIterator.h" +#include "operators/writer/ArrowWriter.h" + +namespace gluten { + +class VeloxArrowWriter : public ArrowWriter { + public: + explicit VeloxArrowWriter(const std::string& path, int64_t batchSize, facebook::velox::memory::MemoryPool* pool); + + std::shared_ptr retrieveColumnarBatch() override; + + private: + int64_t batchSize_; + facebook::velox::memory::MemoryPool* pool_; + std::unique_ptr reader_{nullptr}; +}; + +} // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSource.cc b/cpp/velox/operators/writer/VeloxParquetDataSource.cc index aeec1b4c82d7..07f49c06ed1f 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSource.cc +++ b/cpp/velox/operators/writer/VeloxParquetDataSource.cc @@ -38,7 +38,6 @@ using namespace facebook::velox::common; using namespace facebook::velox::filesystems; namespace gluten { - namespace { const int32_t kGzipWindowBits4k = 12; } diff --git a/cpp/velox/operators/writer/VeloxParquetDataSource.h b/cpp/velox/operators/writer/VeloxParquetDataSource.h index e7428999f02e..a94ab00b9e2a 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSource.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSource.h @@ -37,7 +37,7 @@ #include "velox/connectors/hive/storage_adapters/s3fs/S3Util.h" #endif #ifdef ENABLE_GCS -#include "velox/connectors/hive/storage_adapters/gcs/GCSFileSystem.h" +#include "velox/connectors/hive/storage_adapters/gcs/GcsFileSystem.h" #endif #ifdef ENABLE_HDFS #include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h index 63658cc54ef1..f8f6e5878cff 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceABFS.h @@ -46,9 +46,10 @@ class VeloxParquetDataSourceABFS final : public VeloxParquetDataSource { auto hiveConf = getHiveConfig(std::make_shared( std::unordered_map(sparkConfs))); auto fileSystem = filesystems::getFileSystem(filePath_, hiveConf); - auto* abfsFileSystem = dynamic_cast(fileSystem.get()); + auto* abfsFileSystem = dynamic_cast(fileSystem.get()); sink_ = std::make_unique( abfsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h index 22d4e96efe7f..80c224194c25 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceGCS.h @@ -32,6 +32,7 @@ #include "velox/dwio/common/Options.h" namespace gluten { + class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { public: VeloxParquetDataSourceGCS( @@ -43,9 +44,10 @@ class VeloxParquetDataSourceGCS final : public VeloxParquetDataSource { void initSink(const std::unordered_map& /* sparkConfs */) override { auto fileSystem = filesystems::getFileSystem(filePath_, nullptr); - auto* gcsFileSystem = dynamic_cast(fileSystem.get()); + auto* gcsFileSystem = dynamic_cast(fileSystem.get()); sink_ = std::make_unique( gcsFileSystem->openFileForWrite(filePath_, {{}, sinkPool_.get()}), filePath_); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h b/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h index 053b3da2ff7f..5f61d9145f46 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceHDFS.h @@ -48,4 +48,5 @@ class VeloxParquetDataSourceHDFS final : public VeloxParquetDataSource { sink_ = dwio::common::FileSink::create(filePath_, {.connectorProperties = hiveConf, .pool = sinkPool_.get()}); } }; + } // namespace gluten diff --git a/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h b/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h index 3082f82a9193..788eda19e5d2 100644 --- a/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h +++ b/cpp/velox/operators/writer/VeloxParquetDataSourceS3.h @@ -48,4 +48,5 @@ class VeloxParquetDataSourceS3 final : public VeloxParquetDataSource { sink_ = dwio::common::FileSink::create(filePath_, {.connectorProperties = hiveConf, .pool = sinkPool_.get()}); } }; + } // namespace gluten diff --git a/cpp/velox/shuffle/GlutenByteStream.h b/cpp/velox/shuffle/GlutenByteStream.h index 78ea7b905adc..cc63b730ebca 100644 --- a/cpp/velox/shuffle/GlutenByteStream.h +++ b/cpp/velox/shuffle/GlutenByteStream.h @@ -19,6 +19,7 @@ #include "velox/common/memory/ByteStream.h" namespace facebook::velox { + class GlutenByteInputStream : public ByteInputStream { protected: /// TODO Remove after refactoring SpillInput. @@ -264,4 +265,5 @@ inline int128_t GlutenByteInputStream::read() { readBytes(reinterpret_cast(&value), sizeof(value)); return value; } + } // namespace facebook::velox diff --git a/cpp/velox/shuffle/RadixSort.h b/cpp/velox/shuffle/RadixSort.h index 90921962a17f..aa078ccd06e0 100644 --- a/cpp/velox/shuffle/RadixSort.h +++ b/cpp/velox/shuffle/RadixSort.h @@ -16,6 +16,7 @@ */ #include #include +#include #include #include @@ -26,21 +27,19 @@ namespace gluten { // https://github.com/apache/spark/blob/308669fc301916837bacb7c3ec1ecef93190c094/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/RadixSort.java#L25 class RadixSort { public: - /** - * Sorts a given array of longs using least-significant-digit radix sort. This routine assumes - * you have extra space at the end of the array at least equal to the number of records. The - * sort is destructive and may relocate the data positioned within the array. - * - * @param array array of long elements followed by at least that many empty slots. - * @param numRecords number of data records in the array. - * @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the - * least significant byte. - * @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the - * least significant byte. Must be greater than startByteIndex. - * - * @return The starting index of the sorted data within the given array. We return this instead - * of always copying the data back to position zero for efficiency. - */ + // Sorts a given array of longs using least-significant-digit radix sort. This routine assumes + // you have extra space at the end of the array at least equal to the number of records. The + // sort is destructive and may relocate the data positioned within the array. + // + // @param array array of long elements followed by at least that many empty slots. + // @param numRecords number of data records in the array. + // @param startByteIndex the first byte (in range [0, 7]) to sort each long by, counting from the + // least significant byte. + // @param endByteIndex the last byte (in range [0, 7]) to sort each long by, counting from the + // least significant byte. Must be greater than startByteIndex. + // + // @return The starting index of the sorted data within the given array. We return this instead + // of always copying the data back to position zero for efficiency. static int32_t sort(uint64_t* array, size_t size, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { assert(startByteIndex >= 0 && "startByteIndex should >= 0"); assert(endByteIndex <= 7 && "endByteIndex should <= 7"); @@ -65,17 +64,15 @@ class RadixSort { } private: - /** - * Performs a partial sort by copying data into destination offsets for each byte value at the - * specified byte offset. - * - * @param array array to partially sort. - * @param numRecords number of data records in the array. - * @param counts counts for each byte value. This routine destructively modifies this array. - * @param byteIdx the byte in a long to sort at, counting from the least significant byte. - * @param inIndex the starting index in the array where input data is located. - * @param outIndex the starting index where sorted output data should be written. - */ + // Performs a partial sort by copying data into destination offsets for each byte value at the + // specified byte offset. + // + // @param array array to partially sort. + // @param numRecords number of data records in the array. + // @param counts counts for each byte value. This routine destructively modifies this array. + // @param byteIdx the byte in a long to sort at, counting from the least significant byte. + // @param inIndex the starting index in the array where input data is located. + // @param outIndex the starting index where sorted output data should be written. static void sortAtByte( uint64_t* array, int64_t numRecords, @@ -93,17 +90,15 @@ class RadixSort { } } - /** - * Computes a value histogram for each byte in the given array. - * - * @param array array to count records in. - * @param numRecords number of data records in the array. - * @param startByteIndex the first byte to compute counts for (the prior are skipped). - * @param endByteIndex the last byte to compute counts for. - * - * @return a vector of eight 256-element count arrays, one for each byte starting from the least - * significant byte. If the byte does not need sorting the vector entry will be empty. - */ + // Computes a value histogram for each byte in the given array. + // + // @param array array to count records in. + // @param numRecords number of data records in the array. + // @param startByteIndex the first byte to compute counts for (the prior are skipped). + // @param endByteIndex the last byte to compute counts for. + // + // @return a vector of eight 256-element count arrays, one for each byte starting from the least + // significant byte. If the byte does not need sorting the vector entry will be empty. static std::vector> getCounts(uint64_t* array, int64_t numRecords, int32_t startByteIndex, int32_t endByteIndex) { std::vector> counts; @@ -133,15 +128,13 @@ class RadixSort { return counts; } - /** - * Transforms counts into the proper output offsets for the sort type. - * - * @param counts counts for each byte value. This routine destructively modifies this vector. - * @param numRecords number of data records in the original data array. - * @param outputOffset output offset in bytes from the base array object. - * - * @return the input counts vector. - */ + // Transforms counts into the proper output offsets for the sort type. + // + // @param counts counts for each byte value. This routine destructively modifies this vector. + // @param numRecords number of data records in the original data array. + // @param outputOffset output offset in bytes from the base array object. + // + // @return the input counts vector. static std::vector& transformCountsToOffsets(std::vector& counts, int64_t outputOffset) { assert(counts.size() == 256); diff --git a/cpp/velox/shuffle/VeloxHashShuffleWriter.cc b/cpp/velox/shuffle/VeloxHashShuffleWriter.cc index f04473614291..4cd6630fc3ff 100644 --- a/cpp/velox/shuffle/VeloxHashShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxHashShuffleWriter.cc @@ -725,7 +725,9 @@ arrow::Status VeloxHashShuffleWriter::splitComplexType(const facebook::velox::Ro for (auto& pid : partitionUsed_) { if (rowIndexs[pid].size() != 0) { + auto old = arenas_[pid]->size(); complexTypeData_[pid]->append(rowVector, folly::Range(rowIndexs[pid].data(), rowIndexs[pid].size())); + complexTotalSizeBytes_ += arenas_[pid]->size() - old; } } @@ -853,6 +855,10 @@ uint32_t VeloxHashShuffleWriter::calculatePartitionBufferSize(const facebook::ve VS_PRINT_VECTOR_MAPPING(binaryArrayAvgBytesPerRow); + if (totalInputNumRows_ > 0) { + bytesPerRow += complexTotalSizeBytes_ / totalInputNumRows_; + } + VS_PRINTLF(bytesPerRow); memLimit += cachedPayloadSize(); diff --git a/cpp/velox/shuffle/VeloxHashShuffleWriter.h b/cpp/velox/shuffle/VeloxHashShuffleWriter.h index 121eaf116c00..4ee12a1550d4 100644 --- a/cpp/velox/shuffle/VeloxHashShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxHashShuffleWriter.h @@ -355,6 +355,7 @@ class VeloxHashShuffleWriter : public VeloxShuffleWriter { // Updated for each input RowVector. uint64_t totalInputNumRows_ = 0; std::vector binaryArrayTotalSizeBytes_; + size_t complexTotalSizeBytes_ = 0; // True if input column has null in any processed input RowVector. // In the order of fixed-width columns + binary columns. diff --git a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc index 34796e378e1d..e17ad5e2f713 100644 --- a/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxRssSortShuffleWriter.cc @@ -117,7 +117,6 @@ arrow::Status VeloxRssSortShuffleWriter::write(std::shared_ptr cb } arrow::Status VeloxRssSortShuffleWriter::evictBatch(uint32_t partitionId) { - int64_t rawSize = batch_->size(); bufferOutputStream_->seekp(0); batch_->flush(bufferOutputStream_.get()); auto buffer = bufferOutputStream_->getBuffer(); diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index 32e346d737a3..0407be736a70 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -38,7 +38,6 @@ #include #include -// using namespace facebook; using namespace facebook::velox; namespace gluten { @@ -291,14 +290,14 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, bool hasComplexType, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), @@ -307,13 +306,16 @@ VeloxHashShuffleReaderDeserializer::VeloxHashShuffleReaderDeserializer( isValidityBuffer_(isValidityBuffer), hasComplexType_(hasComplexType), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { if (hasComplexType_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { // Reach EOS. return nullptr; @@ -332,7 +334,8 @@ std::shared_ptr VeloxHashShuffleReaderDeserializer::next() { uint32_t numRows = 0; while (!merged_ || merged_->numRows() < batchSize_) { GLUTEN_ASSIGN_OR_THROW( - arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, decompressTime_)); + arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, memoryPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { reachEos_ = true; break; @@ -372,22 +375,24 @@ VeloxSortShuffleReaderDeserializer::VeloxSortShuffleReaderDeserializer( const std::shared_ptr& codec, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, int64_t& decompressTime) - : in_(std::move(in)), - schema_(schema), + : schema_(schema), codec_(codec), rowType_(rowType), batchSize_(batchSize), arrowPool_(memoryPool), veloxPool_(veloxPool), deserializeTime_(deserializeTime), - decompressTime_(decompressTime) {} + decompressTime_(decompressTime) { + GLUTEN_ASSIGN_OR_THROW(in_, arrow::io::BufferedInputStream::Create(bufferSize, memoryPool, std::move(in))); +} std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { - if (reachEos_) { + if (reachedEos_) { if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -401,10 +406,11 @@ std::shared_ptr VeloxSortShuffleReaderDeserializer::next() { while (cachedRows_ < batchSize_) { uint32_t numRows = 0; GLUTEN_ASSIGN_OR_THROW( - auto arrowBuffers, BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, decompressTime_)); + auto arrowBuffers, + BlockPayload::deserialize(in_.get(), codec_, arrowPool_, numRows, deserializeTime_, decompressTime_)); if (arrowBuffers.empty()) { - reachEos_ = true; + reachedEos_ = true; if (cachedRows_ > 0) { return deserializeToBatch(); } @@ -467,7 +473,8 @@ void VeloxSortShuffleReaderDeserializer::readLargeRow(std::vectorsize(); buffers.emplace_back(std::move(arrowBuffers[0])); @@ -538,6 +545,7 @@ VeloxRssSortShuffleReaderDeserializer::VeloxRssSortShuffleReaderDeserializer( rowType_(rowType), batchSize_(batchSize), veloxCompressionType_(veloxCompressionType), + serde_(getNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)), deserializeTime_(deserializeTime) { constexpr uint64_t kMaxReadBufferSize = (1 << 20) - AlignedBuffer::kPaddedSize; auto buffer = AlignedBuffer::allocate(kMaxReadBufferSize, veloxPool_.get()); @@ -553,7 +561,7 @@ std::shared_ptr VeloxRssSortShuffleReaderDeserializer::next() { ScopedTimer timer(&deserializeTime_); RowVectorPtr rowVector; - VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, &rowVector, &serdeOptions_); + VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, serde_, &rowVector, &serdeOptions_); if (rowVector->size() >= batchSize_) { return std::make_shared(std::move(rowVector)); @@ -561,7 +569,7 @@ std::shared_ptr VeloxRssSortShuffleReaderDeserializer::next() { while (rowVector->size() < batchSize_ && in_->hasNext()) { RowVectorPtr rowVectorTemp; - VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, &rowVectorTemp, &serdeOptions_); + VectorStreamGroup::read(in_.get(), veloxPool_.get(), rowType_, serde_, &rowVectorTemp, &serdeOptions_); rowVector->append(rowVectorTemp.get()); } @@ -574,6 +582,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( const facebook::velox::common::CompressionKind veloxCompressionType, const RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType) @@ -582,6 +591,7 @@ VeloxColumnarBatchDeserializerFactory::VeloxColumnarBatchDeserializerFactory( veloxCompressionType_(veloxCompressionType), rowType_(rowType), batchSize_(batchSize), + bufferSize_(bufferSize), memoryPool_(memoryPool), veloxPool_(veloxPool), shuffleWriterType_(shuffleWriterType) { @@ -598,6 +608,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), &isValidityBuffer_, @@ -611,6 +622,7 @@ std::unique_ptr VeloxColumnarBatchDeserializerFactory::cr codec_, rowType_, batchSize_, + bufferSize_, memoryPool_, veloxPool_.get(), deserializeTime_, diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index f7ff05c5d13e..af35f977127f 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -21,6 +21,7 @@ #include "shuffle/Payload.h" #include "shuffle/ShuffleReader.h" #include "shuffle/VeloxSortShuffleWriter.h" +#include "utils/Timer.h" #include "velox/type/Type.h" #include "velox/vector/ComplexVector.h" @@ -36,6 +37,7 @@ class VeloxHashShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, std::vector* isValidityBuffer, @@ -73,6 +75,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { const std::shared_ptr& codec, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, facebook::velox::memory::MemoryPool* veloxPool, int64_t& deserializeTime, @@ -97,7 +100,7 @@ class VeloxSortShuffleReaderDeserializer final : public ColumnarBatchIterator { std::list> cachedInputs_; uint32_t cachedRows_{0}; - bool reachEos_{false}; + bool reachedEos_{false}; uint32_t rowOffset_{0}; size_t byteOffset_{0}; @@ -125,6 +128,7 @@ class VeloxRssSortShuffleReaderDeserializer : public ColumnarBatchIterator { int32_t rowCount_; int32_t batchSize_; facebook::velox::common::CompressionKind veloxCompressionType_; + facebook::velox::VectorSerde* const serde_; facebook::velox::serializer::presto::PrestoVectorSerde::PrestoOptions serdeOptions_; int64_t& deserializeTime_; std::shared_ptr in_; @@ -138,6 +142,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { const facebook::velox::common::CompressionKind veloxCompressionType, const facebook::velox::RowTypePtr& rowType, int32_t batchSize, + int64_t bufferSize, arrow::MemoryPool* memoryPool, std::shared_ptr veloxPool, ShuffleWriterType shuffleWriterType); @@ -160,6 +165,7 @@ class VeloxColumnarBatchDeserializerFactory : public DeserializerFactory { facebook::velox::common::CompressionKind veloxCompressionType_; facebook::velox::RowTypePtr rowType_; int32_t batchSize_; + int64_t bufferSize_; arrow::MemoryPool* memoryPool_; std::shared_ptr veloxPool_; diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index 69fd6801df54..06b59cf3ebdd 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -21,6 +21,7 @@ #include "shuffle/VeloxSortShuffleWriter.h" namespace gluten { + arrow::Result> VeloxShuffleWriter::create( ShuffleWriterType type, uint32_t numPartitions, @@ -43,4 +44,5 @@ arrow::Result> VeloxShuffleWriter::create( return arrow::Status::Invalid("Unsupported shuffle writer type: ", std::to_string(type)); } } + } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc index f87eaabb56ae..793130a58adc 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.cc @@ -25,8 +25,8 @@ #include "utils/Timer.h" namespace gluten { - namespace { + constexpr uint32_t kMaskLower27Bits = (1 << 27) - 1; constexpr uint64_t kMaskLower40Bits = (1UL << 40) - 1; constexpr uint32_t kPartitionIdStartByteIndex = 5; @@ -44,6 +44,7 @@ uint32_t extractPartitionId(uint64_t compactRowId) { std::pair extractPageNumberAndOffset(uint64_t compactRowId) { return {(compactRowId & kMaskLower40Bits) >> 27, compactRowId & kMaskLower27Bits}; } + } // namespace arrow::Result> VeloxSortShuffleWriter::create( @@ -180,7 +181,7 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr rowSize_.resize(inputRows, *fixedRowSize_); } - uint32_t rowOffset = 0; + facebook::velox::vector_size_t rowOffset = 0; while (rowOffset < inputRows) { auto remainingRows = inputRows - rowOffset; auto rows = maxRowsToInsert(rowOffset, remainingRows); @@ -201,18 +202,23 @@ arrow::Status VeloxSortShuffleWriter::insert(const facebook::velox::RowVectorPtr return arrow::Status::OK(); } -void VeloxSortShuffleWriter::insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows) { +void VeloxSortShuffleWriter::insertRows( + facebook::velox::row::CompactRow& compact, + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t size) { VELOX_CHECK(!pages_.empty()); - for (auto i = offset; i < offset + rows; ++i) { - auto pid = row2Partition_[i]; + std::vector offsets(size); + for (auto i = 0; i < size; ++i) { + auto row = offset + i; + auto pid = row2Partition_[row]; arrayPtr_[offset_++] = toCompactRowId(pid, pageNumber_, pageCursor_); // size(RowSize) | bytes - memcpy(currentPage_ + pageCursor_, &rowSize_[i], sizeof(RowSizeType)); - pageCursor_ += sizeof(RowSizeType); - auto size = row.serialize(i, currentPage_ + pageCursor_); - pageCursor_ += size; + memcpy(currentPage_ + pageCursor_, &rowSize_[row], sizeof(RowSizeType)); + offsets[i] = pageCursor_ + sizeof(RowSizeType); + pageCursor_ += rowSize_[row]; VELOX_DCHECK_LE(pageCursor_, currenPageSize_); } + compact.serialize(offset, size, offsets.data(), currentPage_); } arrow::Status VeloxSortShuffleWriter::maybeSpill(uint32_t nextRows) { @@ -337,19 +343,21 @@ VeloxSortShuffleWriter::evictPartition0(uint32_t partitionId, int32_t numRows, u return arrow::Status::OK(); } -uint32_t VeloxSortShuffleWriter::maxRowsToInsert(uint32_t offset, uint32_t remainingRows) { +facebook::velox::vector_size_t VeloxSortShuffleWriter::maxRowsToInsert( + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t remainingRows) { // Check how many rows can be handled. if (pages_.empty()) { return 0; } auto remainingBytes = pages_.back()->size() - pageCursor_; if (fixedRowSize_) { - return std::min((uint32_t)(remainingBytes / (fixedRowSize_.value())), remainingRows); + return std::min((facebook::velox::vector_size_t)(remainingBytes / (fixedRowSize_.value())), remainingRows); } auto beginIter = rowSizePrefixSum_.begin() + 1 + offset; auto bytesWritten = rowSizePrefixSum_[offset]; auto iter = std::upper_bound(beginIter, rowSizePrefixSum_.end(), remainingBytes + bytesWritten); - return iter - beginIter; + return (facebook::velox::vector_size_t)(iter - beginIter); } void VeloxSortShuffleWriter::acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired) { @@ -442,4 +450,5 @@ void VeloxSortShuffleWriter::updateSpillMetrics(const std::unique_ptrrawSize(); } } + } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxSortShuffleWriter.h b/cpp/velox/shuffle/VeloxSortShuffleWriter.h index 531ed1fe3e76..97c040ba5930 100644 --- a/cpp/velox/shuffle/VeloxSortShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxSortShuffleWriter.h @@ -69,7 +69,10 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { arrow::Status insert(const facebook::velox::RowVectorPtr& vector, int64_t memLimit); - void insertRows(facebook::velox::row::CompactRow& row, uint32_t offset, uint32_t rows); + void insertRows( + facebook::velox::row::CompactRow& compact, + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t size); arrow::Status maybeSpill(uint32_t nextRows); @@ -79,7 +82,9 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { arrow::Status evictPartition0(uint32_t partitionId, int32_t numRows, uint8_t* buffer, int64_t rawLength); - uint32_t maxRowsToInsert(uint32_t offset, uint32_t remainingRows); + facebook::velox::vector_size_t maxRowsToInsert( + facebook::velox::vector_size_t offset, + facebook::velox::vector_size_t remainingRows); void acquireNewBuffer(uint64_t memLimit, uint64_t minSizeRequired); @@ -126,4 +131,5 @@ class VeloxSortShuffleWriter final : public VeloxShuffleWriter { int64_t sortTime_{0}; bool stopped_{false}; }; + } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 01386115b308..5870c4ef9f35 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -73,45 +73,6 @@ EmitInfo getEmitInfo(const ::substrait::RelCommon& relCommon, const core::PlanNo return emitInfo; } -template -// Get the lowest value for numeric type. -T getLowest() { - return std::numeric_limits::lowest(); -} - -// Get the lowest value for string. -template <> -std::string getLowest() { - return ""; -} - -// Get the max value for numeric type. -template -T getMax() { - return std::numeric_limits::max(); -} - -// The max value will be used in BytesRange. Return empty string here instead. -template <> -std::string getMax() { - return ""; -} - -// Substrait function names. -const std::string sIsNotNull = "is_not_null"; -const std::string sIsNull = "is_null"; -const std::string sGte = "gte"; -const std::string sGt = "gt"; -const std::string sLte = "lte"; -const std::string sLt = "lt"; -const std::string sEqual = "equal"; -const std::string sOr = "or"; -const std::string sNot = "not"; - -// Substrait types. -const std::string sI32 = "i32"; -const std::string sI64 = "i64"; - /// @brief Get the input type from both sides of join. /// @param leftNode the plan node of left side. /// @param rightNode the plan node of right side. @@ -179,13 +140,6 @@ RowTypePtr getJoinOutputType( VELOX_FAIL("Output should include left or right columns."); } -// Returns the path vector used to create Subfield. -std::vector> getPath(const std::string& field) { - std::vector> path; - path.push_back(std::make_unique(field)); - return path; -} - } // namespace core::PlanNodePtr SubstraitToVeloxPlanConverter::processEmit( @@ -532,13 +486,43 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } } +std::string makeUuid() { + return boost::lexical_cast(boost::uuids::random_generator()()); +} + +std::string compressionFileNameSuffix(common::CompressionKind kind) { + switch (static_cast(kind)) { + case common::CompressionKind_ZLIB: + return ".zlib"; + case common::CompressionKind_SNAPPY: + return ".snappy"; + case common::CompressionKind_LZO: + return ".lzo"; + case common::CompressionKind_ZSTD: + return ".zstd"; + case common::CompressionKind_LZ4: + return ".lz4"; + case common::CompressionKind_GZIP: + return ".gz"; + case common::CompressionKind_NONE: + default: + return ""; + } +} + std::shared_ptr makeLocationHandle( const std::string& targetDirectory, + dwio::common::FileFormat fileFormat, + common::CompressionKind compression, const std::optional& writeDirectory = std::nullopt, const connector::hive::LocationHandle::TableType& tableType = connector::hive::LocationHandle::TableType::kExisting) { + std::string targetFileName = ""; + if (fileFormat == dwio::common::FileFormat::PARQUET) { + targetFileName = fmt::format("gluten-part-{}{}{}", makeUuid(), compressionFileNameSuffix(compression), ".parquet"); + } return std::make_shared( - targetDirectory, writeDirectory.value_or(targetDirectory), tableType); + targetDirectory, writeDirectory.value_or(targetDirectory), tableType, targetFileName); } std::shared_ptr makeHiveInsertTableHandle( @@ -654,6 +638,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: // Do not hard-code connector ID and allow for connectors other than Hive. static const std::string kHiveConnectorId = "test-hive"; + // Currently only support parquet format. + dwio::common::FileFormat fileFormat = dwio::common::FileFormat::PARQUET; return std::make_shared( nextPlanNodeId(), @@ -667,8 +653,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: inputType->children(), partitionedKey, nullptr /*bucketProperty*/, - makeLocationHandle(writePath), - dwio::common::FileFormat::PARQUET, // Currently only support parquet format. + makeLocationHandle(writePath, fileFormat, compressionCodec), + fileFormat, compressionCodec)), (!partitionedKey.empty()), exec::TableWriteTraits::outputType(nullptr), @@ -1037,7 +1023,11 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan( // Handle if all sorting keys are also used as partition keys. return std::make_shared( - nextPlanNodeId(), partitionKeys, rowNumberColumnName, (int32_t)windowGroupLimitRel.limit(), childNode); + nextPlanNodeId(), + partitionKeys, + rowNumberColumnName, + static_cast(windowGroupLimitRel.limit()), + childNode); } return std::make_shared( @@ -1046,10 +1036,55 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan( sortingKeys, sortingOrders, rowNumberColumnName, - (int32_t)windowGroupLimitRel.limit(), + static_cast(windowGroupLimitRel.limit()), childNode); } +core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::SetRel& setRel) { + switch (setRel.op()) { + case ::substrait::SetRel_SetOp::SetRel_SetOp_SET_OP_UNION_ALL: { + std::vector children; + for (int32_t i = 0; i < setRel.inputs_size(); ++i) { + const auto& input = setRel.inputs(i); + children.push_back(toVeloxPlan(input)); + } + GLUTEN_CHECK(!children.empty(), "At least one source is required for Velox LocalPartition"); + + // Velox doesn't allow different field names in schemas of LocalPartitionNode's children. + // Add project nodes to unify the schemas. + const RowTypePtr outRowType = asRowType(children[0]->outputType()); + std::vector outNames; + for (int32_t colIdx = 0; colIdx < outRowType->size(); ++colIdx) { + const auto name = outRowType->childAt(colIdx)->name(); + outNames.push_back(name); + } + + std::vector projectedChildren; + for (int32_t i = 0; i < children.size(); ++i) { + const auto& child = children[i]; + const RowTypePtr& childRowType = child->outputType(); + std::vector expressions; + for (int32_t colIdx = 0; colIdx < outNames.size(); ++colIdx) { + const auto fa = + std::make_shared(childRowType->childAt(colIdx), childRowType->nameOf(colIdx)); + const auto cast = std::make_shared(outRowType->childAt(colIdx), fa, false); + expressions.push_back(cast); + } + auto project = std::make_shared(nextPlanNodeId(), outNames, expressions, child); + projectedChildren.push_back(project); + } + return std::make_shared( + nextPlanNodeId(), + core::LocalPartitionNode::Type::kGather, + false, + std::make_shared(), + projectedChildren); + } + default: + throw GlutenException("Unsupported SetRel op: " + std::to_string(setRel.op())); + } +} + core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::SortRel& sortRel) { auto childNode = convertSingleInput<::substrait::SortRel>(sortRel); auto [sortingKeys, sortingOrders] = processSortField(sortRel.sorts(), childNode->outputType()); @@ -1092,14 +1127,18 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::FetchRel& fetchRel) { auto childNode = convertSingleInput<::substrait::FetchRel>(fetchRel); return std::make_shared( - nextPlanNodeId(), (int32_t)fetchRel.offset(), (int32_t)fetchRel.count(), false /*isPartial*/, childNode); + nextPlanNodeId(), + static_cast(fetchRel.offset()), + static_cast(fetchRel.count()), + false /*isPartial*/, + childNode); } core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::TopNRel& topNRel) { auto childNode = convertSingleInput<::substrait::TopNRel>(topNRel); auto [sortingKeys, sortingOrders] = processSortField(topNRel.sorts(), childNode->outputType()); return std::make_shared( - nextPlanNodeId(), sortingKeys, sortingOrders, (int32_t)topNRel.n(), false /*isPartial*/, childNode); + nextPlanNodeId(), sortingKeys, sortingOrders, static_cast(topNRel.n()), false /*isPartial*/, childNode); } core::PlanNodePtr SubstraitToVeloxPlanConverter::constructValueStreamNode( @@ -1129,8 +1168,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::constructValueStreamNode( VELOX_CHECK_LT(streamIdx, inputIters_.size(), "Could not find stream index {} in input iterator list.", streamIdx); iterator = inputIters_[streamIdx]; } - auto valueStream = std::make_unique(pool_, iterator, outputType); - auto node = std::make_shared(nextPlanNodeId(), outputType, std::move(valueStream)); + auto node = std::make_shared(nextPlanNodeId(), outputType, std::move(iterator)); auto splitInfo = std::make_shared(); splitInfo->isStream = true; @@ -1191,37 +1229,10 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, connector::hive::SubfieldFilters{}, nullptr); } else { - // Flatten the conditions connected with 'and'. - std::vector<::substrait::Expression_ScalarFunction> scalarFunctions; - std::vector<::substrait::Expression_SingularOrList> singularOrLists; - std::vector<::substrait::Expression_IfThen> ifThens; - flattenConditions(readRel.filter(), scalarFunctions, singularOrLists, ifThens); - - // The vector's subscript stands for the column index. - std::vector rangeRecorders(veloxTypeList.size()); - - // Separate the filters to be two parts. The subfield part can be - // pushed down. - std::vector<::substrait::Expression_ScalarFunction> subfieldFunctions; - std::vector<::substrait::Expression_ScalarFunction> remainingFunctions; - std::vector<::substrait::Expression_SingularOrList> subfieldOrLists; - std::vector<::substrait::Expression_SingularOrList> remainingOrLists; - - separateFilters( - rangeRecorders, - scalarFunctions, - subfieldFunctions, - remainingFunctions, - singularOrLists, - subfieldOrLists, - remainingOrLists, - veloxTypeList, - splitInfo->format); - - // Create subfield filters based on the constructed filter info map. - auto subfieldFilters = createSubfieldFilters(colNameList, veloxTypeList, subfieldFunctions, subfieldOrLists); - // Connect the remaining filters with 'and'. - auto remainingFilter = connectWithAnd(colNameList, veloxTypeList, remainingFunctions, remainingOrLists, ifThens); + connector::hive::SubfieldFilters subfieldFilters; + auto names = colNameList; + auto types = veloxTypeList; + auto remainingFilter = exprConverter_->toVeloxExpr(readRel.filter(), ROW(std::move(names), std::move(types))); tableHandle = std::make_shared( kHiveConnectorId, "hive_table", filterPushdownEnabled, std::move(subfieldFilters), remainingFilter); @@ -1333,6 +1344,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: return toVeloxPlan(rel.write()); } else if (rel.has_windowgrouplimit()) { return toVeloxPlan(rel.windowgrouplimit()); + } else if (rel.has_set()) { + return toVeloxPlan(rel.set()); } else { VELOX_NYI("Substrait conversion not supported for Rel."); } @@ -1387,39 +1400,6 @@ void SubstraitToVeloxPlanConverter::constructFunctionMap(const ::substrait::Plan exprConverter_ = std::make_unique(pool_, functionMap_); } -void SubstraitToVeloxPlanConverter::flattenConditions( - const ::substrait::Expression& substraitFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens) { - auto typeCase = substraitFilter.rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kScalarFunction: { - const auto& sFunc = substraitFilter.scalar_function(); - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, sFunc.function_reference()); - // TODO: Only and relation is supported here. - if (SubstraitParser::getNameBeforeDelimiter(filterNameSpec) == "and") { - for (const auto& sCondition : sFunc.arguments()) { - flattenConditions(sCondition.value(), scalarFunctions, singularOrLists, ifThens); - } - } else { - scalarFunctions.emplace_back(sFunc); - } - break; - } - case ::substrait::Expression::RexTypeCase::kSingularOrList: { - singularOrLists.emplace_back(substraitFilter.singular_or_list()); - break; - } - case ::substrait::Expression::RexTypeCase::kIfThen: { - ifThens.emplace_back(substraitFilter.if_then()); - break; - } - default: - VELOX_NYI("GetFlatConditions not supported for type '{}'", std::to_string(typeCase)); - } -} - std::string SubstraitToVeloxPlanConverter::findFuncSpec(uint64_t id) { return SubstraitParser::findFunctionSpec(functionMap_, id); } @@ -1482,878 +1462,6 @@ void SubstraitToVeloxPlanConverter::extractJoinKeys( } } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists) { - // The vector's subscript stands for the column index. - std::vector columnToFilterInfo(inputTypeList.size()); - - // Process scalarFunctions. - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - - if (filterName == sNot) { - VELOX_CHECK(scalarFunction.arguments().size() == 1); - auto expr = scalarFunction.arguments()[0].value(); - if (expr.has_scalar_function()) { - // Set its child to filter info with reverse enabled. - setFilterInfo(expr.scalar_function(), inputTypeList, columnToFilterInfo, true); - } else if (expr.has_singular_or_list()) { - auto singularOrList = expr.singular_or_list(); - setFilterInfo(singularOrList, columnToFilterInfo, true); - } else { - VELOX_NYI("Only support push down Not with scalar function or In."); - } - } else if (filterName == sOr) { - VELOX_CHECK(scalarFunction.arguments().size() == 2); - VELOX_CHECK(std::all_of( - scalarFunction.arguments().cbegin(), - scalarFunction.arguments().cend(), - [](const ::substrait::FunctionArgument& arg) { - return arg.value().has_scalar_function() || arg.value().has_singular_or_list(); - })); - - // Set the children functions to filter info. They should be - // effective to the same field. - for (const auto& arg : scalarFunction.arguments()) { - const auto& expr = arg.value(); - if (expr.has_scalar_function()) { - setFilterInfo(arg.value().scalar_function(), inputTypeList, columnToFilterInfo); - } else if (expr.has_singular_or_list()) { - setFilterInfo(expr.singular_or_list(), columnToFilterInfo); - } else { - VELOX_NYI("Scalar function or SingularOrList expected."); - } - } - } else { - setFilterInfo(scalarFunction, inputTypeList, columnToFilterInfo); - } - } - - // Process singularOrLists. - for (const auto& list : singularOrLists) { - setFilterInfo(list, columnToFilterInfo); - } - - return mapToFilters(inputNameList, inputTypeList, columnToFilterInfo); -} - -bool SubstraitToVeloxPlanConverter::fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex) { - if (arguments.size() == 1) { - if (arguments[0].value().has_selection()) { - // Only field exists. - return SubstraitParser::parseReferenceSegment(arguments[0].value().selection().direct_reference(), fieldIndex); - } else { - return false; - } - } - - if (arguments.size() != 2) { - // Not the field and literal combination. - return false; - } - bool fieldExists = false; - bool literalExists = false; - for (const auto& param : arguments) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - if (!SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), fieldIndex)) { - return false; - } - fieldExists = true; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - literalExists = true; - break; - } - default: - break; - } - } - // Whether the field and literal both exist. - return fieldExists && literalExists; -} - -bool SubstraitToVeloxPlanConverter::childrenFunctionsOnSameField( - const ::substrait::Expression_ScalarFunction& function) { - // Get the column indices of the children functions. - std::vector colIndices; - for (const auto& arg : function.arguments()) { - if (arg.value().has_scalar_function()) { - const auto& scalarFunction = arg.value().scalar_function(); - for (const auto& param : scalarFunction.arguments()) { - if (param.value().has_selection()) { - const auto& field = param.value().selection(); - VELOX_CHECK(field.has_direct_reference()); - uint32_t colIdx; - if (!SubstraitParser::parseReferenceSegment(field.direct_reference(), colIdx)) { - return false; - } - colIndices.emplace_back(colIdx); - } - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - colIndices.emplace_back(getColumnIndexFromSingularOrList(singularOrList)); - } else { - return false; - } - } - - if (std::all_of(colIndices.begin(), colIndices.end(), [&](uint32_t idx) { return idx == colIndices[0]; })) { - // All indices are the same. - return true; - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx) { - // Condtions can be pushed down. - static const std::unordered_set supportedFunctions = {sIsNotNull, sIsNull, sGte, sGt, sLte, sLt, sEqual}; - - bool canPushdown = false; - if (supportedFunctions.find(filterName) != supportedFunctions.end() && - fieldOrWithLiteral(scalarFunction.arguments(), fieldIdx)) { - // The arg should be field or field with literal. - canPushdown = true; - } - return canPushdown; -} - -bool SubstraitToVeloxPlanConverter::canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - VELOX_CHECK(scalarFunction.arguments().size() == 1, "Only one arg is expected for Not."); - const auto& notArg = scalarFunction.arguments()[0]; - if (notArg.value().has_singular_or_list()) { - auto singularOrList = notArg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList)) { - return false; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - return rangeRecorders.at(colIdx).setInRange(); - } else if (notArg.value().has_scalar_function()) { - auto argFunction = - SubstraitParser::findFunctionSpec(functionMap_, notArg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(argFunction); - - static const std::unordered_set supportedNotFunctions = {sGte, sGt, sLte, sLt, sEqual}; - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(notArg.value().scalar_function().arguments(), fieldIdx); - - return ( - supportedNotFunctions.find(functionName) != supportedNotFunctions.end() && isFieldOrWithLiteral && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(functionName, true /*reverse*/)); - } - return false; -} - -bool SubstraitToVeloxPlanConverter::canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders) { - // OR Conditon whose children functions are on different columns is not - // supported to be pushed down. - if (!childrenFunctionsOnSameField(scalarFunction)) { - return false; - } - - static const std::unordered_set supportedOrFunctions = {sIsNotNull, sGte, sGt, sLte, sLt, sEqual}; - - for (const auto& arg : scalarFunction.arguments()) { - if (arg.value().has_scalar_function()) { - auto nameSpec = - SubstraitParser::findFunctionSpec(functionMap_, arg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - uint32_t fieldIdx; - bool isFieldOrWithLiteral = fieldOrWithLiteral(arg.value().scalar_function().arguments(), fieldIdx); - if (supportedOrFunctions.find(functionName) == supportedOrFunctions.end() || !isFieldOrWithLiteral || - !rangeRecorders.at(fieldIdx).setCertainRangeForFunction( - functionName, false /*reverse*/, true /*forOrRelation*/)) { - // The arg should be field or field with literal. - return false; - } - } else if (arg.value().has_singular_or_list()) { - const auto& singularOrList = arg.value().singular_or_list(); - if (!canPushdownSingularOrList(singularOrList, true)) { - return false; - } - uint32_t fieldIdx = getColumnIndexFromSingularOrList(singularOrList); - // Disable IN pushdown for int-like types. - if (!rangeRecorders.at(fieldIdx).setInRange(true /*forOrRelation*/)) { - return false; - } - } else { - // Or relation betweeen other expressions is not supported to be pushded - // down currently. - return false; - } - } - return true; -} - -void SubstraitToVeloxPlanConverter::separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format) { - for (const auto& singularOrList : singularOrLists) { - if (!canPushdownSingularOrList(singularOrList)) { - remainingOrLists.emplace_back(singularOrList); - continue; - } - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - if (rangeRecorders.at(colIdx).setInRange()) { - subfieldOrLists.emplace_back(singularOrList); - } else { - remainingOrLists.emplace_back(singularOrList); - } - } - - for (const auto& scalarFunction : scalarFunctions) { - auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); - // Add all decimal filters to remaining functions because their pushdown are not supported. - if (format == dwio::common::FileFormat::ORC && scalarFunction.arguments().size() > 0) { - auto value = scalarFunction.arguments().at(0).value(); - if (value.has_selection()) { - uint32_t fieldIndex; - bool parsed = SubstraitParser::parseReferenceSegment(value.selection().direct_reference(), fieldIndex); - if (!parsed || (!veloxTypeList.empty() && veloxTypeList.at(fieldIndex)->isDecimal())) { - remainingFunctions.emplace_back(scalarFunction); - continue; - } - } - } - - // Check whether NOT and OR functions can be pushed down. - // If yes, the scalar function will be added into the subfield functions. - if (filterName == sNot) { - if (canPushdownNot(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else if (filterName == sOr) { - if (canPushdownOr(scalarFunction, rangeRecorders)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } else { - // Check if the condition is supported to be pushed down. - uint32_t fieldIdx; - if (canPushdownFunction(scalarFunction, filterName, fieldIdx) && - rangeRecorders.at(fieldIdx).setCertainRangeForFunction(filterName)) { - subfieldFunctions.emplace_back(scalarFunction); - } else { - remainingFunctions.emplace_back(scalarFunction); - } - } - } -} - -bool SubstraitToVeloxPlanConverter::RangeRecorder::setCertainRangeForFunction( - const std::string& functionName, - bool reverse, - bool forOrRelation) { - if (functionName == sLt || functionName == sLte) { - if (reverse) { - return setLeftBound(forOrRelation); - } else { - return setRightBound(forOrRelation); - } - } else if (functionName == sGt || functionName == sGte) { - if (reverse) { - return setRightBound(forOrRelation); - } else { - return setLeftBound(forOrRelation); - } - } else if (functionName == sEqual) { - if (reverse) { - // Not equal means lt or gt. - return setMultiRange(); - } else { - return setLeftBound(forOrRelation) && setRightBound(forOrRelation); - } - } else if (functionName == sOr) { - if (reverse) { - // Not supported. - return false; - } else { - return setMultiRange(); - } - } else if (functionName == sIsNotNull) { - if (reverse) { - // Not supported. - return false; - } else { - // Is not null can always coexist with the other range. - return true; - } - } else if (functionName == sIsNull) { - if (reverse) { - return setCertainRangeForFunction(sIsNotNull, false, forOrRelation); - } else { - return setIsNull(); - } - } else { - return false; - } -} - -void SubstraitToVeloxPlanConverter::setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnFilterInfo, - bool reverse) { - if (filterName == sIsNotNull) { - if (reverse) { - columnFilterInfo.setNull(); - } else { - columnFilterInfo.forbidsNull(); - } - } else if (filterName == sIsNull) { - if (reverse) { - columnFilterInfo.forbidsNull(); - } else { - columnFilterInfo.setNull(); - } - } else if (filterName == sGte) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, true); - } else { - columnFilterInfo.setLower(literalVariant, false); - } - } else if (filterName == sGt) { - if (reverse) { - columnFilterInfo.setUpper(literalVariant, false); - } else { - columnFilterInfo.setLower(literalVariant, true); - } - } else if (filterName == sLte) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, true); - } else { - columnFilterInfo.setUpper(literalVariant, false); - } - } else if (filterName == sLt) { - if (reverse) { - columnFilterInfo.setLower(literalVariant, false); - } else { - columnFilterInfo.setUpper(literalVariant, true); - } - } else if (filterName == sEqual) { - if (reverse) { - columnFilterInfo.setNotValue(literalVariant); - } else { - columnFilterInfo.setLower(literalVariant, false); - columnFilterInfo.setUpper(literalVariant, false); - } - } else { - VELOX_NYI("setColumnFilterInfo not supported for filter name '{}'", filterName); - } -} - -template -variant getVariantFromLiteral(const ::substrait::Expression::Literal& literal) { - using LitT = typename facebook::velox::TypeTraits::NativeType; - return variant(SubstraitParser::getLiteralValue(literal)); -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse) { - auto nameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); - - // Extract the column index and column bound from the scalar function. - std::optional colIdx; - std::optional<::substrait::Expression_Literal> substraitLit; - std::vector typeCases; - - for (const auto& param : scalarFunction.arguments()) { - auto typeCase = param.value().rex_type_case(); - switch (typeCase) { - case ::substrait::Expression::RexTypeCase::kSelection: { - typeCases.emplace_back("kSelection"); - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(param.value().selection().direct_reference(), index), - "Failed to parse the column index from the selection."); - colIdx = index; - break; - } - case ::substrait::Expression::RexTypeCase::kLiteral: { - typeCases.emplace_back("kLiteral"); - substraitLit = param.value().literal(); - break; - } - default: - VELOX_NYI("Substrait conversion not supported for arg type '{}'", std::to_string(typeCase)); - } - } - - static const std::unordered_map functionRevertMap = { - {sLt, sGt}, {sGt, sLt}, {sGte, sLte}, {sLte, sGte}}; - - // Handle the case where literal is before the variable in a binary function, e.g. "123 < q1". - if (typeCases.size() > 1 && (typeCases[0] == "kLiteral" && typeCases[1] == "kSelection")) { - auto x = functionRevertMap.find(functionName); - if (x != functionRevertMap.end()) { - // Change the function name: lt => gt, gt => lt, gte => lte, lte => gte. - functionName = x->second; - } - } - - if (!colIdx.has_value()) { - VELOX_NYI("Column index is expected in subfield filters creation."); - } - - // Set the extracted bound to the specific column. - uint32_t colIdxVal = colIdx.value(); - std::optional val; - - auto inputType = inputTypeList[colIdxVal]; - switch (inputType->kind()) { - case TypeKind::TINYINT: - case TypeKind::SMALLINT: - case TypeKind::INTEGER: - case TypeKind::BIGINT: - case TypeKind::REAL: - case TypeKind::DOUBLE: - case TypeKind::BOOLEAN: - case TypeKind::VARCHAR: - case TypeKind::HUGEINT: - if (substraitLit) { - auto kind = inputType->kind(); - val = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(getVariantFromLiteral, kind, substraitLit.value()); - } - break; - case TypeKind::ARRAY: - case TypeKind::MAP: - case TypeKind::ROW: - // Doing nothing here can let filter IsNotNull still work. - break; - default: - VELOX_NYI("Subfield filters creation not supported for input type '{}' in setFilterInfo", inputType->toString()); - } - - setColumnFilterInfo(functionName, val, columnToFilterInfo[colIdxVal], reverse); -} - -template -void SubstraitToVeloxPlanConverter::createNotEqualFilter( - variant notVariant, - bool nullAllowed, - std::vector>& colFilters) { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - // Value > lower - std::unique_ptr lowerFilter; - if constexpr (std::is_same_v) { - if (notVariant.value() < getMax()) { - lowerFilter = std::make_unique( - notVariant.value() + 1 /*lower*/, getMax() /*upper*/, nullAllowed); - } - } else { - lowerFilter = std::make_unique( - notVariant.value() /*lower*/, - false /*lowerUnbounded*/, - true /*lowerExclusive*/, - getMax() /*upper*/, - true /*upperUnbounded*/, - false /*upperExclusive*/, - nullAllowed); - } - - // Value < upper - std::unique_ptr upperFilter; - if constexpr (std::is_same_v) { - if (getLowest() < notVariant.value()) { - upperFilter = std::make_unique( - getLowest() /*lower*/, notVariant.value() - 1 /*upper*/, nullAllowed); - } - } else { - upperFilter = std::make_unique( - getLowest() /*lower*/, - true /*lowerUnbounded*/, - false /*lowerExclusive*/, - notVariant.value() /*upper*/, - false /*upperUnbounded*/, - true /*upperExclusive*/, - nullAllowed); - } - - // To avoid overlap of BigintMultiRange, keep this appending order to make sure lower bound of one range is less than - // the upper bounds of others. - if (upperFilter) { - colFilters.emplace_back(std::move(upperFilter)); - } - if (lowerFilter) { - colFilters.emplace_back(std::move(lowerFilter)); - } -} - -template -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) {} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for small int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - // Use bigint values for tiny int type. - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - // Use the matched type to get value from variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createNegatedBigintValues(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = common::createBigintValues(values, nullAllowed); - } -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters) { - std::vector values; - values.reserve(variants.size()); - for (const auto& variant : variants) { - std::string value = variant.value(); - values.emplace_back(value); - } - if (negated) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(values, nullAllowed); - } -} - -template -void SubstraitToVeloxPlanConverter::setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters) { - using MultiRangeType = typename RangeTraits::MultiRangeType; - - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters[0]); - } else if (colFilters.size() > 1) { - // BigintMultiRange should have been sorted - if (colFilters[0]->kind() == common::FilterKind::kBigintRange) { - std::sort(colFilters.begin(), colFilters.end(), [](const auto& a, const auto& b) { - return dynamic_cast(a.get())->lower() < - dynamic_cast(b.get())->lower(); - }); - } - if constexpr (std::is_same_v) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } -} - -template -void SubstraitToVeloxPlanConverter::constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters) { - if (!filterInfo.isInitialized()) { - return; - } - - bool nullAllowed = filterInfo.nullAllowed_; - bool isNull = filterInfo.isNull_; - bool existIsNullAndIsNotNull = filterInfo.forbidsNullSet_ && filterInfo.isNullSet_; - uint32_t rangeSize = std::max(filterInfo.lowerBounds_.size(), filterInfo.upperBounds_.size()); - - if constexpr (KIND == facebook::velox::TypeKind::HUGEINT) { - // TODO: open it when the Velox's modification is ready. - VELOX_NYI("constructSubfieldFilters not support for HUGEINT type"); - } else if constexpr (KIND == facebook::velox::TypeKind::BOOLEAN) { - // Handle bool type filters. - // Not equal. - if (filterInfo.notValue_) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!filterInfo.notValue_.value().value(), nullAllowed); - } else if (filterInfo.notValues_.size() > 0) { - std::set notValues; - for (auto v : filterInfo.notValues_) { - notValues.emplace(v.value()); - } - if (notValues.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(!(*notValues.begin()), nullAllowed); - } else { - // if there are more than one distinct value in NOT IN list, the filter should be AlwaysFalse - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } - } else if (rangeSize == 0) { - // IsNull/IsNotNull. - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } else { - // Equal. - auto value = filterInfo.lowerBounds_[0].value().value(); - VELOX_CHECK(value == filterInfo.upperBounds_[0].value().value(), "invalid state of bool equal"); - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(value, nullAllowed); - } - } else if constexpr ( - KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP || - KIND == facebook::velox::TypeKind::ROW) { - // Only IsNotNull and IsNull are supported for complex types. - VELOX_CHECK_EQ(rangeSize, 0, "Only IsNotNull and IsNull are supported for complex type."); - if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported for input type '{}'.", inputType->toString()); - } - } else { - using NativeType = typename RangeTraits::NativeType; - using RangeType = typename RangeTraits::RangeType; - using MultiRangeType = typename RangeTraits::MultiRangeType; - - // Handle 'in' filter. - if (filterInfo.values_.size() > 0) { - // To filter out null is a default behaviour of Spark IN expression. - nullAllowed = false; - setInFilter(filterInfo.values_, nullAllowed, false, inputName, filters); - // Currently, In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after IN filter."); - VELOX_CHECK(filterInfo.notValues_.size() == 0, "Not in cannot be supported after IN filter."); - return; - } - - // Handle not in filter. - if (filterInfo.notValues_.size() > 0) { - setInFilter(filterInfo.notValues_, filterInfo.nullAllowed_, true, inputName, filters); - // Currently, NOT In cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after NOT IN filter."); - VELOX_CHECK(!filterInfo.notValue_.has_value(), "Not equal cannot be supported after NOT IN filter."); - return; - } - - // Construct the Filters. - std::vector> colFilters; - - // Handle not(equal) filter. - if (filterInfo.notValue_) { - variant notVariant = filterInfo.notValue_.value(); - createNotEqualFilter(notVariant, filterInfo.nullAllowed_, colFilters); - // Currently, Not-equal cannot coexist with other filter conditions - // due to multirange is in 'OR' relation but 'AND' is needed. - VELOX_CHECK(rangeSize == 0, "LowerBounds or upperBounds conditons cannot be supported after not-equal filter."); - if constexpr (std::is_same_v) { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); - } - } else { - if (colFilters.size() == 1) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(colFilters.front()); - } else { - filters[common::Subfield(std::move(getPath(inputName)))] = - std::make_unique(std::move(colFilters), nullAllowed); - } - } - return; - } - - // Handle null filtering. - if (rangeSize == 0) { - // handle is not null and is null exists at same time - if (existIsNullAndIsNotNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::move(std::make_unique()); - } else if (!nullAllowed) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else if (isNull) { - filters[common::Subfield(std::move(getPath(inputName)))] = std::make_unique(); - } else { - VELOX_NYI("Only IsNotNull and IsNull are supported in constructSubfieldFilters when no other filter ranges."); - } - return; - } - - NativeType lowerBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - lowerBound = DecimalUtil::kShortDecimalMin; - } else { - lowerBound = getLowest(); - } - } else { - lowerBound = getLowest(); - } - - NativeType upperBound; - if constexpr (KIND == facebook::velox::TypeKind::BIGINT) { - if (inputType->isShortDecimal()) { - upperBound = DecimalUtil::kShortDecimalMax; - } else { - upperBound = getMax(); - } - } else { - upperBound = getMax(); - } - - [[maybe_unused]] bool lowerUnbounded = true; - [[maybe_unused]] bool upperUnbounded = true; - bool lowerExclusive = false; - bool upperExclusive = false; - - // Handle other filter ranges. - for (uint32_t idx = 0; idx < rangeSize; idx++) { - if (idx < filterInfo.lowerBounds_.size() && filterInfo.lowerBounds_[idx]) { - lowerUnbounded = false; - variant lowerVariant = filterInfo.lowerBounds_[idx].value(); - lowerBound = lowerVariant.value(); - lowerExclusive = filterInfo.lowerExclusives_[idx]; - } - - if (idx < filterInfo.upperBounds_.size() && filterInfo.upperBounds_[idx]) { - upperUnbounded = false; - variant upperVariant = filterInfo.upperBounds_[idx].value(); - upperBound = upperVariant.value(); - upperExclusive = filterInfo.upperExclusives_[idx]; - } - - std::unique_ptr filter; - if constexpr (std::is_same_v) { - filter = std::move(std::make_unique( - lowerExclusive ? lowerBound + 1 : lowerBound, upperExclusive ? upperBound - 1 : upperBound, nullAllowed)); - } else { - filter = std::move(std::make_unique( - lowerBound, lowerUnbounded, lowerExclusive, upperBound, upperUnbounded, upperExclusive, nullAllowed)); - } - - colFilters.emplace_back(std::move(filter)); - } - - // Set the SubfieldFilter. - setSubfieldFilter(std::move(colFilters), inputName, filterInfo.nullAllowed_, filters); - } -} - bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& substraitPlan) { for (const auto& sExtension : substraitPlan.extensions()) { if (!sExtension.has_extension_type()) { @@ -2368,199 +1476,4 @@ bool SubstraitToVeloxPlanConverter::checkTypeExtension(const ::substrait::Plan& return true; } -connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo) { - // Construct the subfield filters based on the filter info map. - connector::hive::SubfieldFilters filters; - for (uint32_t colIdx = 0; colIdx < inputNameList.size(); colIdx++) { - if (columnToFilterInfo[colIdx].isInitialized()) { - auto inputType = inputTypeList[colIdx]; - if (inputType->isDate()) { - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - continue; - } - switch (inputType->kind()) { - case TypeKind::TINYINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::SMALLINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::INTEGER: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BIGINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::REAL: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::DOUBLE: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::BOOLEAN: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::VARCHAR: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::HUGEINT: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ARRAY: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::MAP: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - case TypeKind::ROW: - constructSubfieldFilters( - colIdx, inputNameList[colIdx], inputType, columnToFilterInfo[colIdx], filters); - break; - default: - VELOX_NYI( - "Subfield filters creation not supported for input type '{}' in mapToFilters", inputType->toString()); - } - } - } - - return filters; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens) { - if (scalarFunctions.size() == 0 && singularOrLists.size() == 0 && ifThens.size() == 0) { - return nullptr; - } - auto inputType = ROW(std::move(inputNameList), std::move(inputTypeList)); - - // Filter for scalar functions. - std::vector allFilters; - for (auto scalar : scalarFunctions) { - auto filter = exprConverter_->toVeloxExpr(scalar, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto orList : singularOrLists) { - auto filter = exprConverter_->toVeloxExpr(orList, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - - for (auto ifThen : ifThens) { - auto filter = exprConverter_->toVeloxExpr(ifThen, inputType); - if (filter != nullptr) { - allFilters.emplace_back(filter); - } - } - VELOX_CHECK_GT(allFilters.size(), 0, "One filter should be valid."); - core::TypedExprPtr andFilter = allFilters[0]; - for (auto i = 1; i < allFilters.size(); i++) { - andFilter = connectWithAnd(andFilter, allFilters[i]); - } - - return andFilter; -} - -core::TypedExprPtr SubstraitToVeloxPlanConverter::connectWithAnd( - core::TypedExprPtr leftExpr, - core::TypedExprPtr rightExpr) { - std::vector params; - params.reserve(2); - params.emplace_back(leftExpr); - params.emplace_back(rightExpr); - return std::make_shared(BOOLEAN(), std::move(params), "and"); -} - -bool SubstraitToVeloxPlanConverter::canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Check whether the value is field. - bool hasField = singularOrList.value().has_selection(); - const auto& options = singularOrList.options(); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - auto type = option.literal().literal_type_case(); - // Only BigintValues and BytesValues are supported. - if (type != ::substrait::Expression_Literal::LiteralTypeCase::kI32 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kI64 && - type != ::substrait::Expression_Literal::LiteralTypeCase::kString) { - return false; - } - - // BigintMultiRange can only accept BigintRange, so disableIntLike is set to - // true for OR pushdown of int-like types. - if (disableIntLike && - (type == ::substrait::Expression_Literal::LiteralTypeCase::kI32 || - type == ::substrait::Expression_Literal::LiteralTypeCase::kI64)) { - return false; - } - } - return hasField; -} - -uint32_t SubstraitToVeloxPlanConverter::getColumnIndexFromSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList) { - // Get the column index. - ::substrait::Expression_FieldReference selection; - if (singularOrList.value().has_scalar_function()) { - selection = singularOrList.value().scalar_function().arguments()[0].value().selection(); - } else if (singularOrList.value().has_selection()) { - selection = singularOrList.value().selection(); - } else { - VELOX_FAIL("Unsupported type in IN pushdown."); - } - uint32_t index; - VELOX_CHECK( - SubstraitParser::parseReferenceSegment(selection.direct_reference(), index), - "Failed to parse column index from SingularOrList."); - return index; -} - -void SubstraitToVeloxPlanConverter::setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse) { - VELOX_CHECK(singularOrList.options_size() > 0, "At least one option is expected."); - // Get the column index. - uint32_t colIdx = getColumnIndexFromSingularOrList(singularOrList); - - // Get the value list. - const auto& options = singularOrList.options(); - std::vector variants; - variants.reserve(options.size()); - for (const auto& option : options) { - VELOX_CHECK(option.has_literal(), "Literal is expected as option."); - variants.emplace_back(exprConverter_->toVeloxExpr(option.literal())->value()); - } - // Set the value list to filter info. - if (!reverse) { - columnToFilterInfo[colIdx].setValues(variants); - } else { - columnToFilterInfo[colIdx].setNotValues(variants); - } -} - } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 0e892469d098..6121923df787 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -84,6 +84,9 @@ class SubstraitToVeloxPlanConverter { /// Used to convert Substrait WindowGroupLimitRel into Velox PlanNode. core::PlanNodePtr toVeloxPlan(const ::substrait::WindowGroupLimitRel& windowGroupLimitRel); + /// Used to convert Substrait SetRel into Velox PlanNode. + core::PlanNodePtr toVeloxPlan(const ::substrait::SetRel& setRel); + /// Used to convert Substrait JoinRel into Velox PlanNode. core::PlanNodePtr toVeloxPlan(const ::substrait::JoinRel& joinRel); @@ -215,354 +218,13 @@ class SubstraitToVeloxPlanConverter { /// if output order is 'kDriect'. core::PlanNodePtr processEmit(const ::substrait::RelCommon& relCommon, const core::PlanNodePtr& noEmitNode); - /// Multiple conditions are connected to a binary tree structure with - /// the relation key words, including AND, OR, and etc. Currently, only - /// AND is supported. This function is used to extract all the Substrait - /// conditions in the binary tree structure into a vector. - void flattenConditions( - const ::substrait::Expression& sFilter, - std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_IfThen>& ifThens); - /// Check the Substrait type extension only has one unknown extension. static bool checkTypeExtension(const ::substrait::Plan& substraitPlan); - /// Range filter recorder for a field is used to make sure only the conditions - /// that can coexist for this field being pushed down with a range filter. - class RangeRecorder { - public: - /// Set the existence of values range and returns whether this condition can - /// coexist with existing conditions for one field. Conditions in OR - /// relation can coexist with each other. - bool setInRange(bool forOrRelation = false) { - if (forOrRelation) { - return true; - } - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - inRange_ = true; - return true; - } - - /// Set the existence of left bound and returns whether it can coexist with - /// existing conditions for this field. - bool setLeftBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!rightBound_) - leftBound_ = true; - return !rightBound_; - } - if (leftBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - leftBound_ = true; - return true; - } - - /// Set the existence of right bound and returns whether it can coexist with - /// existing conditions for this field. - bool setRightBound(bool forOrRelation = false) { - if (forOrRelation) { - if (!leftBound_) - rightBound_ = true; - return !leftBound_; - } - if (rightBound_ || inRange_ || multiRange_ || isNull_) { - return false; - } - rightBound_ = true; - return true; - } - - /// Set the existence of multi-range and returns whether it can coexist with - /// existing conditions for this field. - bool setMultiRange() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_ || isNull_) { - return false; - } - multiRange_ = true; - return true; - } - - /// Set the existence of IsNull and returns whether it can coexist with - /// existing conditions for this field. - bool setIsNull() { - if (inRange_ || multiRange_ || leftBound_ || rightBound_) { - return false; - } - isNull_ = true; - return true; - } - - /// Set certain existence according to function name and returns whether it - /// can coexist with existing conditions for this field. - bool setCertainRangeForFunction(const std::string& functionName, bool reverse = false, bool forOrRelation = false); - - private: - /// The existence of values range. - bool inRange_ = false; - - /// The existence of left bound. - bool leftBound_ = false; - - /// The existence of right bound. - bool rightBound_ = false; - - /// The existence of multi-range. - bool multiRange_ = false; - - /// The existence of IsNull. - bool isNull_ = false; - }; - - /// Filter info for a column used in filter push down. - class FilterInfo { - public: - // Null is not allowed. - void forbidsNull() { - nullAllowed_ = false; - if (!initialized_) { - initialized_ = true; - } - forbidsNullSet_ = true; - } - - // Only null is allowed. - void setNull() { - isNull_ = true; - nullAllowed_ = true; - if (!initialized_) { - initialized_ = true; - } - isNullSet_ = true; - } - - // Return the initialization status. - bool isInitialized() const { - return initialized_; - } - - // Add a lower bound to the range. Multiple lower bounds are - // regarded to be in 'or' relation. - void setLower(const std::optional& left, bool isExclusive) { - lowerBounds_.emplace_back(left); - lowerExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Add a upper bound to the range. Multiple upper bounds are - // regarded to be in 'or' relation. - void setUpper(const std::optional& right, bool isExclusive) { - upperBounds_.emplace_back(right); - upperExclusives_.emplace_back(isExclusive); - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'in' expression. - void setValues(const std::vector& values) { - for (const auto& value : values) { - values_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Set a value for the not(equal) condition. - void setNotValue(const std::optional& notValue) { - notValue_ = notValue; - if (!initialized_) { - initialized_ = true; - } - } - - // Set a list of values to be used in the push down of 'not in' expression. - void setNotValues(const std::vector& notValues) { - for (const auto& value : notValues) { - notValues_.emplace_back(value); - } - if (!initialized_) { - initialized_ = true; - } - } - - // Whether this filter map is initialized. - bool initialized_ = false; - - bool nullAllowed_ = false; - bool isNull_ = false; - bool forbidsNullSet_ = false; - bool isNullSet_ = false; - - // If true, left bound will be exclusive. - std::vector lowerExclusives_; - - // If true, right bound will be exclusive. - std::vector upperExclusives_; - - // A value should not be equal to. - std::optional notValue_ = std::nullopt; - - // The lower bounds in 'or' relation. - std::vector> lowerBounds_; - - // The upper bounds in 'or' relation. - std::vector> upperBounds_; - - // The list of values used in 'in' expression. - std::vector values_; - - // The list of values should not be equal to. - std::vector notValues_; - }; - /// Returns unique ID to use for plan node. Produces sequential numbers /// starting from zero. std::string nextPlanNodeId(); - /// Returns whether the args of a scalar function being field or - /// field with literal. If yes, extract and set the field index. - static bool fieldOrWithLiteral( - const ::google::protobuf::RepeatedPtrField<::substrait::FunctionArgument>& arguments, - uint32_t& fieldIndex); - - /// Separate the functions to be two parts: - /// subfield functions to be handled by the subfieldFilters in HiveConnector, - /// and remaining functions to be handled by the remainingFilter in - /// HiveConnector. - void separateFilters( - std::vector& rangeRecorders, - const std::vector<::substrait::Expression_ScalarFunction>& scalarFunctions, - std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - std::vector<::substrait::Expression_SingularOrList>& subfieldrOrLists, - std::vector<::substrait::Expression_SingularOrList>& remainingrOrLists, - const std::vector& veloxTypeList, - const dwio::common::FileFormat& format); - - /// Returns whether a function can be pushed down. - static bool canPushdownFunction( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::string& filterName, - uint32_t& fieldIdx); - - /// Returns whether a NOT function can be pushed down. - bool canPushdownNot( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a OR function can be pushed down. - bool canPushdownOr( - const ::substrait::Expression_ScalarFunction& scalarFunction, - std::vector& rangeRecorders); - - /// Returns whether a SingularOrList can be pushed down. - static bool canPushdownSingularOrList( - const ::substrait::Expression_SingularOrList& singularOrList, - bool disableIntLike = false); - - /// Check whether the children functions of this scalar function have the same - /// column index. Curretly used to check whether the two chilren functions of - /// 'or' expression are effective on the same column. - static bool childrenFunctionsOnSameField(const ::substrait::Expression_ScalarFunction& function); - - /// Extract the scalar function, and set the filter info for different types - /// of columns. If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_ScalarFunction& scalarFunction, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and set it to the filter info map. - /// If reverse is true, the opposite filter info will be set. - void setFilterInfo( - const ::substrait::Expression_SingularOrList& singularOrList, - std::vector& columnToFilterInfo, - bool reverse = false); - - /// Extract SingularOrList and returns the field index. - static uint32_t getColumnIndexFromSingularOrList(const ::substrait::Expression_SingularOrList&); - - /// Set the filter info for a column base on the information - /// extracted from filter condition. - static void setColumnFilterInfo( - const std::string& filterName, - std::optional literalVariant, - FilterInfo& columnToFilterInfo, - bool reverse); - - /// Create a multirange to specify the filter 'x != notValue' with: - /// x > notValue or x < notValue. - template - void createNotEqualFilter(variant notVariant, bool nullAllowed, std::vector>& colFilters); - - /// Create a values range to handle (not) in filter. - /// variants: the list of values extracted from the (not) in expression. - // negated: false for IN filter, true for NOT IN filter. - /// inputName: the column input name. - template - void setInFilter( - const std::vector& variants, - bool nullAllowed, - bool negated, - const std::string& inputName, - connector::hive::SubfieldFilters& filters); - - /// Set the constructed filters into SubfieldFilters. - /// The FilterType is used to distinguish BigintRange and - /// Filter (the base class). This is needed because BigintMultiRange - /// can only accept the unique ptr of BigintRange as parameter. - template - void setSubfieldFilter( - std::vector> colFilters, - const std::string& inputName, - bool nullAllowed, - connector::hive::SubfieldFilters& filters); - - /// Create the subfield filter based on the constructed filter info. - /// inputName: the input name of a column. - template - void constructSubfieldFilters( - uint32_t colIdx, - const std::string& inputName, - const TypePtr& inputType, - const FilterInfo& filterInfo, - connector::hive::SubfieldFilters& filters); - - /// Construct subfield filters according to the pre-set map of filter info. - connector::hive::SubfieldFilters mapToFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - std::vector& columnToFilterInfo); - - /// Convert subfield functions into subfieldFilters to - /// be used in Hive Connector. - connector::hive::SubfieldFilters createSubfieldFilters( - const std::vector& inputNameList, - const std::vector& inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& subfieldFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists); - - /// Connect all remaining functions with 'and' relation - /// for the use of remaingFilter in Hive Connector. - core::TypedExprPtr connectWithAnd( - std::vector inputNameList, - std::vector inputTypeList, - const std::vector<::substrait::Expression_ScalarFunction>& remainingFunctions, - const std::vector<::substrait::Expression_SingularOrList>& singularOrLists, - const std::vector<::substrait::Expression_IfThen>& ifThens); - - /// Connect the left and right expressions with 'and' relation. - core::TypedExprPtr connectWithAnd(core::TypedExprPtr leftExpr, core::TypedExprPtr rightExpr); - /// Used to convert AggregateRel into Velox plan node. /// The output of child node will be used as the input of Aggregation. std::shared_ptr toVeloxAgg( diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index 3b74caf8ba5a..153db7097002 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -22,15 +22,14 @@ #include "TypeUtils.h" #include "udf/UdfLoader.h" #include "utils/Common.h" -#include "velox/core/ExpressionEvaluator.h" #include "velox/exec/Aggregate.h" #include "velox/expression/Expr.h" #include "velox/expression/SignatureBinder.h" namespace gluten { - namespace { -static const char* extractFileName(const char* file) { + +const char* extractFileName(const char* file) { return strrchr(file, '/') ? strrchr(file, '/') + 1 : file; } @@ -53,13 +52,13 @@ static const char* extractFileName(const char* file) { __FUNCTION__, \ reason)) -static const std::unordered_set kRegexFunctions = { +const std::unordered_set kRegexFunctions = { "regexp_extract", "regexp_extract_all", "regexp_replace", "rlike"}; -static const std::unordered_set kBlackList = { +const std::unordered_set kBlackList = { "split_part", "factorial", "concat_ws", @@ -70,32 +69,59 @@ static const std::unordered_set kBlackList = { "approx_percentile", "get_array_struct_fields", "map_from_arrays"}; - } // namespace -bool SubstraitToVeloxPlanValidator::validateInputTypes( +bool SubstraitToVeloxPlanValidator::parseVeloxType( const ::substrait::extensions::AdvancedExtension& extension, - std::vector& types) { + TypePtr& out) { + ::substrait::Type substraitType; // The input type is wrapped in enhancement. if (!extension.has_enhancement()) { LOG_VALIDATION_MSG("Input type is not wrapped in enhancement."); return false; } const auto& enhancement = extension.enhancement(); - ::substrait::Type inputType; - if (!enhancement.UnpackTo(&inputType)) { + if (!enhancement.UnpackTo(&substraitType)) { LOG_VALIDATION_MSG("Enhancement can't be unpacked to inputType."); return false; } - if (!inputType.has_struct_()) { - LOG_VALIDATION_MSG("Input type has no struct."); + + out = SubstraitParser::parseType(substraitType); + return true; +} + +bool SubstraitToVeloxPlanValidator::flattenVeloxType1(const TypePtr& type, std::vector& out) { + if (type->kind() != TypeKind::ROW) { + LOG_VALIDATION_MSG("Type is not a RowType."); + return false; + } + auto rowType = std::dynamic_pointer_cast(type); + if (!rowType) { + LOG_VALIDATION_MSG("Failed to cast to RowType."); return false; } + for (const auto& field : rowType->children()) { + out.emplace_back(field); + } + return true; +} - // Get the input types. - const auto& sTypes = inputType.struct_().types(); - for (const auto& sType : sTypes) { - types.emplace_back(SubstraitParser::parseType(sType)); +bool SubstraitToVeloxPlanValidator::flattenVeloxType2(const TypePtr& type, std::vector>& out) { + if (type->kind() != TypeKind::ROW) { + LOG_VALIDATION_MSG("Type is not a RowType."); + return false; + } + auto rowType = std::dynamic_pointer_cast(type); + if (!rowType) { + LOG_VALIDATION_MSG("Failed to cast to RowType."); + return false; + } + for (const auto& field : rowType->children()) { + std::vector inner; + if (!flattenVeloxType1(field, inner)) { + return false; + } + out.emplace_back(inner); } return true; } @@ -341,10 +367,11 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WriteRel& writeR } // Validate input data type. + TypePtr inputRowType; std::vector types; if (writeRel.has_named_table()) { const auto& extension = writeRel.named_table().advanced_extension(); - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input type validation in WriteRel."); return false; } @@ -380,12 +407,12 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WriteRel& writeR } bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FetchRel& fetchRel) { - RowTypePtr rowType = nullptr; // Get and validate the input types from extension. if (fetchRel.has_advanced_extension()) { const auto& extension = fetchRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in FetchRel."); return false; } @@ -396,7 +423,6 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FetchRel& fetchR for (auto colIdx = 0; colIdx < types.size(); colIdx++) { names.emplace_back(SubstraitParser::makeNodeName(inputPlanNodeId, colIdx)); } - rowType = std::make_shared(std::move(names), std::move(types)); } if (fetchRel.offset() < 0 || fetchRel.count() < 0) { @@ -412,8 +438,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::TopNRel& topNRel // Get and validate the input types from extension. if (topNRel.has_advanced_extension()) { const auto& extension = topNRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in TopNRel."); return false; } @@ -457,8 +484,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::GenerateRel& gen return false; } const auto& extension = generateRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in GenerateRel."); return false; } @@ -487,8 +515,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::ExpandRel& expan // Get and validate the input types from extension. if (expandRel.has_advanced_extension()) { const auto& extension = expandRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Unsupported input types in ExpandRel."); return false; } @@ -571,8 +600,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo return false; } const auto& extension = windowRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in WindowRel."); return false; } @@ -680,7 +710,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo LOG_VALIDATION_MSG("in windowRel, the sorting key in Sort Operator only support field."); return false; } - exec::ExprSet exprSet({std::move(expression)}, execCtx_); + exec::ExprSet exprSet1({std::move(expression)}, execCtx_); } } @@ -699,8 +729,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowGroupLimit return false; } const auto& extension = windowGroupLimitRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in WindowGroupLimitRel."); return false; } @@ -750,13 +781,61 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowGroupLimit LOG_VALIDATION_MSG("in windowGroupLimitRel, the sorting key in Sort Operator only support field."); return false; } - exec::ExprSet exprSet({std::move(expression)}, execCtx_); + exec::ExprSet exprSet1({std::move(expression)}, execCtx_); } } return true; } +bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SetRel& setRel) { + switch (setRel.op()) { + case ::substrait::SetRel_SetOp::SetRel_SetOp_SET_OP_UNION_ALL: { + for (int32_t i = 0; i < setRel.inputs_size(); ++i) { + const auto& input = setRel.inputs(i); + if (!validate(input)) { + LOG_VALIDATION_MSG("ProjectRel input"); + return false; + } + } + if (!setRel.has_advanced_extension()) { + LOG_VALIDATION_MSG("Input types are expected in SetRel."); + return false; + } + const auto& extension = setRel.advanced_extension(); + TypePtr inputRowType; + std::vector> childrenTypes; + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType2(inputRowType, childrenTypes)) { + LOG_VALIDATION_MSG("Validation failed for input types in SetRel."); + return false; + } + std::vector childrenRowTypes; + for (auto i = 0; i < childrenTypes.size(); ++i) { + auto& types = childrenTypes.at(i); + std::vector names; + names.reserve(types.size()); + for (auto colIdx = 0; colIdx < types.size(); colIdx++) { + names.emplace_back(SubstraitParser::makeNodeName(i, colIdx)); + } + childrenRowTypes.push_back(std::make_shared(std::move(names), std::move(types))); + } + + for (auto i = 1; i < childrenRowTypes.size(); ++i) { + if (!(childrenRowTypes[i]->equivalent(*childrenRowTypes[0]))) { + LOG_VALIDATION_MSG( + "All sources of the Set operation must have the same output type: " + childrenRowTypes[i]->toString() + + " vs. " + childrenRowTypes[0]->toString()); + return false; + } + } + return true; + } + default: + LOG_VALIDATION_MSG("Unsupported SetRel op: " + std::to_string(setRel.op())); + return false; + } +} + bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SortRel& sortRel) { if (sortRel.has_input() && !validate(sortRel.input())) { return false; @@ -769,8 +848,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::SortRel& sortRel } const auto& extension = sortRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in SortRel."); return false; } @@ -822,8 +902,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::ProjectRel& proj return false; } const auto& extension = projectRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in ProjectRel."); return false; } @@ -865,8 +946,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FilterRel& filte return false; } const auto& extension = filterRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in FilterRel."); return false; } @@ -938,8 +1020,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::JoinRel& joinRel } const auto& extension = joinRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { LOG_VALIDATION_MSG("Validation failed for input types in JoinRel."); return false; } @@ -991,8 +1074,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::CrossRel& crossR } const auto& extension = crossRel.advanced_extension(); + TypePtr inputRowType; std::vector types; - if (!validateInputTypes(extension, types)) { + if (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types)) { logValidateMsg("Native validation failed due to: Validation failed for input types in CrossRel"); return false; } @@ -1070,11 +1154,13 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::AggregateRel& ag // Validate input types. if (aggRel.has_advanced_extension()) { + TypePtr inputRowType; std::vector types; const auto& extension = aggRel.advanced_extension(); // Aggregate always has advanced extension for streaming aggregate optimization, // but only some of them have enhancement for validation. - if (extension.has_enhancement() && !validateInputTypes(extension, types)) { + if (extension.has_enhancement() && + (!parseVeloxType(extension, inputRowType) || !flattenVeloxType1(inputRowType, types))) { LOG_VALIDATION_MSG("Validation failed for input types in AggregateRel."); return false; } @@ -1266,7 +1352,10 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::Rel& rel) { return validate(rel.write()); } else if (rel.has_windowgrouplimit()) { return validate(rel.windowgrouplimit()); + } else if (rel.has_set()) { + return validate(rel.set()); } else { + LOG_VALIDATION_MSG("Unsupported relation type: " + rel.GetTypeName()); return false; } } diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h index 1fe174928fd9..0c8d882ca031 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h @@ -61,6 +61,9 @@ class SubstraitToVeloxPlanValidator { /// Used to validate whether the computing of this WindowGroupLimit is supported. bool validate(const ::substrait::WindowGroupLimitRel& windowGroupLimitRel); + /// Used to validate whether the computing of this Set is supported. + bool validate(const ::substrait::SetRel& setRel); + /// Used to validate whether the computing of this Aggregation is supported. bool validate(const ::substrait::AggregateRel& aggRel); @@ -103,9 +106,17 @@ class SubstraitToVeloxPlanValidator { std::vector validateLog_; - /// Used to get types from advanced extension and validate them. - bool validateInputTypes(const ::substrait::extensions::AdvancedExtension& extension, std::vector& types); + /// Used to get types from advanced extension and validate them, then convert to a Velox type that has arbitrary + /// levels of nesting. + bool parseVeloxType(const ::substrait::extensions::AdvancedExtension& extension, TypePtr& out); + + /// Flattens a Velox type with single level of nesting into a std::vector of child types. + bool flattenVeloxType1(const TypePtr& type, std::vector& out); + + /// Flattens a Velox type with two level of nesting into a dual-nested std::vector of child types. + bool flattenVeloxType2(const TypePtr& type, std::vector>& out); + /// Validate aggregate rel. bool validateAggRelFunctionType(const ::substrait::AggregateRel& substraitAgg); /// Validate the round scalar function. diff --git a/cpp/velox/substrait/TypeUtils.h b/cpp/velox/substrait/TypeUtils.h index b2aaf725789d..20dd33031d71 100644 --- a/cpp/velox/substrait/TypeUtils.h +++ b/cpp/velox/substrait/TypeUtils.h @@ -22,6 +22,7 @@ using namespace facebook::velox; namespace gluten { + #ifndef RANGETRAITS_H #define RANGETRAITS_H diff --git a/cpp/velox/substrait/VariantToVectorConverter.cc b/cpp/velox/substrait/VariantToVectorConverter.cc index f8c9c11f0f37..f33633b91e50 100644 --- a/cpp/velox/substrait/VariantToVectorConverter.cc +++ b/cpp/velox/substrait/VariantToVectorConverter.cc @@ -67,4 +67,5 @@ VectorPtr setVectorFromVariantsByKind( VectorPtr setVectorFromVariants(const TypePtr& type, const std::vector& values, memory::MemoryPool* pool) { return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(setVectorFromVariantsByKind, type->kind(), values, type, pool); } + } // namespace gluten diff --git a/cpp/velox/substrait/VeloxToSubstraitPlan.cc b/cpp/velox/substrait/VeloxToSubstraitPlan.cc index 19259f81f629..44fbb01b39a6 100644 --- a/cpp/velox/substrait/VeloxToSubstraitPlan.cc +++ b/cpp/velox/substrait/VeloxToSubstraitPlan.cc @@ -20,7 +20,6 @@ #include "utils/Exception.h" namespace gluten { - namespace { struct AggregateCompanion { diff --git a/cpp/velox/symbols.map b/cpp/velox/symbols.map index 525faf3526a1..427560c8f122 100644 --- a/cpp/velox/symbols.map +++ b/cpp/velox/symbols.map @@ -9,6 +9,6 @@ JNI_OnLoad; JNI_OnUnload; local: - # Hide symbols of static dependencies + # Hide all other symbols except the above global symbols. *; }; diff --git a/cpp/velox/tests/BufferOutputStreamTest.cc b/cpp/velox/tests/BufferOutputStreamTest.cc index 0e16f8c87769..b9ea62fd7b56 100644 --- a/cpp/velox/tests/BufferOutputStreamTest.cc +++ b/cpp/velox/tests/BufferOutputStreamTest.cc @@ -24,6 +24,7 @@ using namespace facebook::velox; namespace gluten { + class BufferOutputStreamTest : public ::testing::Test, public test::VectorTestBase { protected: // Velox requires the mem manager to be instanced. @@ -69,4 +70,5 @@ TEST_F(BufferOutputStreamTest, outputStream) { // We expect dropping the stream frees the backing memory. EXPECT_EQ(0, veloxPool_->usedBytes()); } + } // namespace gluten diff --git a/cpp/velox/tests/CMakeLists.txt b/cpp/velox/tests/CMakeLists.txt index c4cf6bfb0756..ba90e45076cf 100644 --- a/cpp/velox/tests/CMakeLists.txt +++ b/cpp/velox/tests/CMakeLists.txt @@ -26,11 +26,9 @@ function(add_velox_test TEST_EXEC) message(FATAL_ERROR "No sources specified for test ${TEST_NAME}") endif() add_executable(${TEST_EXEC} ${SOURCES} ${VELOX_TEST_COMMON_SRCS}) - target_include_directories( - ${TEST_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox ${CMAKE_SOURCE_DIR}/src - ${VELOX_BUILD_PATH}/_deps/duckdb-src/src/include) - target_link_libraries(${TEST_EXEC} velox GTest::gtest GTest::gtest_main - google::glog) + target_include_directories(${TEST_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox + ${CMAKE_SOURCE_DIR}/src) + target_link_libraries(${TEST_EXEC} velox GTest::gtest GTest::gtest_main) gtest_discover_tests(${TEST_EXEC} DISCOVERY_MODE PRE_TEST) endfunction() @@ -62,5 +60,5 @@ add_velox_test(runtime_test SOURCES RuntimeTest.cc) add_velox_test(velox_memory_test SOURCES MemoryManagerTest.cc) add_velox_test(buffer_outputstream_test SOURCES BufferOutputStreamTest.cc) if(BUILD_EXAMPLES) - add_velox_test(MyUdfTest SOURCES MyUdfTest.cc) + add_velox_test(my_udf_test SOURCES MyUdfTest.cc) endif() diff --git a/cpp/velox/tests/FunctionTest.cc b/cpp/velox/tests/FunctionTest.cc index b55b64ba9811..c149b2db38fc 100644 --- a/cpp/velox/tests/FunctionTest.cc +++ b/cpp/velox/tests/FunctionTest.cc @@ -34,6 +34,7 @@ using namespace facebook::velox; using namespace facebook::velox::test; namespace gluten { + class FunctionTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -212,4 +213,5 @@ TEST_F(FunctionTest, sigToTypes) { ASSERT_TRUE(types[0]->childAt(2)->isDecimal()); ASSERT_EQ(types[0]->childAt(3)->kind(), TypeKind::BOOLEAN); } + } // namespace gluten diff --git a/cpp/velox/tests/MemoryManagerTest.cc b/cpp/velox/tests/MemoryManagerTest.cc index 8794f9d3af2a..85df9eed49e4 100644 --- a/cpp/velox/tests/MemoryManagerTest.cc +++ b/cpp/velox/tests/MemoryManagerTest.cc @@ -21,6 +21,7 @@ #include "velox/common/base/tests/GTestUtils.h" namespace gluten { + using namespace facebook::velox; class MockAllocationListener : public gluten::AllocationListener { @@ -273,7 +274,8 @@ void MockSparkTaskMemoryManager::release(uint64_t bytes) { class MockMemoryReclaimer : public facebook::velox::memory::MemoryReclaimer { public: - explicit MockMemoryReclaimer(std::vector& buffs, int32_t size) : buffs_(buffs), size_(size) {} + explicit MockMemoryReclaimer(std::vector& buffs, int32_t size) + : facebook::velox::memory::MemoryReclaimer(0), buffs_(buffs), size_(size) {} bool reclaimableBytes(const memory::MemoryPool& pool, uint64_t& reclaimableBytes) const override { uint64_t total = 0; @@ -398,4 +400,5 @@ TEST_F(MultiMemoryManagerTest, spill) { ASSERT_EQ(tmm.currentBytes(), 0); } + } // namespace gluten diff --git a/cpp/velox/tests/MyUdfTest.cc b/cpp/velox/tests/MyUdfTest.cc index c9849d67d010..8e4f1e7dec7e 100644 --- a/cpp/velox/tests/MyUdfTest.cc +++ b/cpp/velox/tests/MyUdfTest.cc @@ -17,12 +17,13 @@ #include #include "udf/UdfLoader.h" -#include "velox/expression/VectorFunction.h" +#include "velox/expression/SimpleFunctionRegistry.h" #include "velox/functions/prestosql/tests/utils/FunctionBaseTest.h" #include "velox/parse/TypeResolver.h" using namespace facebook::velox::functions::test; using namespace facebook::velox; + class MyUdfTest : public FunctionBaseTest { protected: static void SetUpTestCase() { @@ -35,16 +36,7 @@ class MyUdfTest : public FunctionBaseTest { }; TEST_F(MyUdfTest, hivestringstring) { - auto map = facebook::velox::exec::vectorFunctionFactories(); - const std::string candidate = {"org.apache.spark.sql.hive.execution.UDFStringString"}; - ASSERT(map.withRLock([&candidate](auto& self) -> bool { - auto iter = self.find(candidate); - std::unordered_map values; - const facebook::velox::core::QueryConfig config(std::move(values)); - iter->second.factory( - candidate, - {facebook::velox::exec::VectorFunctionArg{facebook::velox::VARCHAR()}, - facebook::velox::exec::VectorFunctionArg{facebook::velox::VARCHAR()}}, - config) != nullptr; - });) -} \ No newline at end of file + const std::string name = "org.apache.spark.sql.hive.execution.UDFStringString"; + const core::QueryConfig config({}); + EXPECT_EQ(TypeKind::VARCHAR, exec::simpleFunctions().resolveFunction(name, {VARCHAR(), VARCHAR()})->type()->kind()); +} diff --git a/cpp/velox/tests/RuntimeTest.cc b/cpp/velox/tests/RuntimeTest.cc index 4b5bcdf81919..b4944d92054b 100644 --- a/cpp/velox/tests/RuntimeTest.cc +++ b/cpp/velox/tests/RuntimeTest.cc @@ -103,6 +103,10 @@ class DummyRuntime final : public Runtime { throw GlutenException("Not yet implemented"); } + std::shared_ptr createArrowWriter(const std::string& path) override { + throw GlutenException("Not yet implemented"); + } + private: class DummyResultIterator : public ColumnarBatchIterator { public: diff --git a/cpp/velox/tests/SparkFunctionTest.cc b/cpp/velox/tests/SparkFunctionTest.cc index 2105b155e3db..ae5e7c48d8ab 100644 --- a/cpp/velox/tests/SparkFunctionTest.cc +++ b/cpp/velox/tests/SparkFunctionTest.cc @@ -22,6 +22,7 @@ using namespace facebook::velox::functions::sparksql::test; using namespace facebook::velox; + class SparkFunctionTest : public SparkFunctionBaseTest { public: SparkFunctionTest() { diff --git a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc index 06d4ea019572..cccc619a86ff 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanConversionTest.cc @@ -35,6 +35,7 @@ using namespace facebook::velox::connector::hive; using namespace facebook::velox::exec; namespace gluten { + class Substrait2VeloxPlanConversionTest : public exec::test::HiveConnectorTestBase { protected: std::vector> makeSplits( @@ -256,12 +257,7 @@ TEST_F(Substrait2VeloxPlanConversionTest, ifthenTest) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n " - "-- TableScan[0][table: hive_table, range filters: [(hd_demo_sk, Filter(IsNotNull, deterministic, null not allowed))," - " (hd_vehicle_count, BigintRange: [1, 9223372036854775807] no nulls)], remaining filter: " - "(and(or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))," - "if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE," - "cast \"hd_vehicle_count\" as DOUBLE),1.2))))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(and(and(and(isnotnull(\"hd_vehicle_count\"),or(equalto(\"hd_buy_potential\",\">10000\"),equalto(\"hd_buy_potential\",\"unknown\"))),greaterthan(\"hd_vehicle_count\",0)),if(greaterthan(\"hd_vehicle_count\",0),greaterthan(divide(cast \"hd_dep_count\" as DOUBLE,cast \"hd_vehicle_count\" as DOUBLE),1.2))),isnotnull(\"hd_demo_sk\")))] -> n0_0:BIGINT, n0_1:VARCHAR, n0_2:BIGINT, n0_3:BIGINT\n", planNode->toString(true, true)); } @@ -277,8 +273,8 @@ TEST_F(Substrait2VeloxPlanConversionTest, filterUpper) { // Convert to Velox PlanNode. auto planNode = planConverter_->toVeloxPlan(substraitPlan, std::vector<::substrait::ReadRel_LocalFiles>{split}); ASSERT_EQ( - "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, range filters: " - "[(key, BigintRange: [-2147483648, 2] no nulls)]] -> n0_0:INTEGER\n", + "-- Project[1][expressions: ] -> \n -- TableScan[0][table: hive_table, remaining filter: (and(isnotnull(\"key\"),lessthan(\"key\",3)))] -> n0_0:INTEGER\n", planNode->toString(true, true)); } + } // namespace gluten diff --git a/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc b/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc index 0a957f038f8d..3f90c865df16 100644 --- a/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc +++ b/cpp/velox/tests/Substrait2VeloxPlanValidatorTest.cc @@ -35,6 +35,7 @@ using namespace facebook::velox::connector::hive; using namespace facebook::velox::exec; namespace gluten { + class Substrait2VeloxPlanValidatorTest : public exec::test::HiveConnectorTestBase { protected: bool validatePlan(std::string file) { @@ -64,4 +65,5 @@ TEST_F(Substrait2VeloxPlanValidatorTest, group) { ASSERT_FALSE(validatePlan(substraitPlan)); } + } // namespace gluten diff --git a/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc b/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc index 75099db95976..b5a131790a64 100644 --- a/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc +++ b/cpp/velox/tests/Substrait2VeloxValuesNodeConversionTest.cc @@ -32,6 +32,7 @@ using namespace facebook::velox::exec; using namespace facebook::velox::exec::test; namespace gluten { + class Substrait2VeloxValuesNodeConversionTest : public OperatorTestBase {}; // SELECT * FROM tmp @@ -57,4 +58,5 @@ TEST_F(Substrait2VeloxValuesNodeConversionTest, valuesNode) { createDuckDbTable({expectedData}); assertQuery(veloxPlan, "SELECT * FROM tmp"); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxBatchResizerTest.cc b/cpp/velox/tests/VeloxBatchResizerTest.cc index aecd52f927cc..fc741e015468 100644 --- a/cpp/velox/tests/VeloxBatchResizerTest.cc +++ b/cpp/velox/tests/VeloxBatchResizerTest.cc @@ -82,4 +82,5 @@ TEST_F(VeloxBatchResizerTest, sanity) { checkResize(100, 200, {5, 900, 700}, {5, 200, 200, 200, 200, 100, 200, 200, 200, 100}); ASSERT_ANY_THROW(checkResize(0, 0, {}, {})); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index ffa6f032ac44..333bb047f900 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -19,15 +19,17 @@ #include "memory/ArrowMemoryPool.h" #include "memory/VeloxColumnarBatch.h" -#include "memory/VeloxMemoryManager.h" #include "operators/serializer/VeloxColumnarBatchSerializer.h" #include "utils/VeloxArrowUtils.h" #include "velox/vector/arrow/Bridge.h" #include "velox/vector/tests/utils/VectorTestBase.h" +#include + using namespace facebook::velox; namespace gluten { + class VeloxColumnarBatchSerializerTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { diff --git a/cpp/velox/tests/VeloxColumnarBatchTest.cc b/cpp/velox/tests/VeloxColumnarBatchTest.cc index ba66afb40fdf..3ad975751539 100644 --- a/cpp/velox/tests/VeloxColumnarBatchTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchTest.cc @@ -61,4 +61,5 @@ TEST_F(VeloxColumnarBatchTest, flattenTruncatedVector) { auto batchOfMap = std::make_shared(inputOfMap); ASSERT_NO_THROW(batchOfMap->getFlattenedRowVector()); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxColumnarToRowTest.cc b/cpp/velox/tests/VeloxColumnarToRowTest.cc index 3adacdda9d51..85bdf5b9ecda 100644 --- a/cpp/velox/tests/VeloxColumnarToRowTest.cc +++ b/cpp/velox/tests/VeloxColumnarToRowTest.cc @@ -27,6 +27,7 @@ using namespace facebook; using namespace facebook::velox; namespace gluten { + class VeloxColumnarToRowTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -92,4 +93,5 @@ TEST_F(VeloxColumnarToRowTest, Buffer_int64_int64_with_null) { }; testRowBufferAddr(vector, expectArr, sizeof(expectArr)); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxRowToColumnarTest.cc b/cpp/velox/tests/VeloxRowToColumnarTest.cc index 0d11dd4acbc9..937b8b80b6cd 100644 --- a/cpp/velox/tests/VeloxRowToColumnarTest.cc +++ b/cpp/velox/tests/VeloxRowToColumnarTest.cc @@ -26,6 +26,7 @@ using namespace facebook; using namespace facebook::velox; namespace gluten { + class VeloxRowToColumnarTest : public ::testing::Test, public test::VectorTestBase { protected: static void SetUpTestCase() { @@ -141,4 +142,5 @@ TEST_F(VeloxRowToColumnarTest, timestamp) { }); testRowVectorEqual(vector); } + } // namespace gluten diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index ddcc30ff1ce9..e760a469b1e7 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -35,7 +35,6 @@ using namespace arrow; using namespace arrow::ipc; namespace gluten { - namespace { facebook::velox::RowVectorPtr takeRows( @@ -728,4 +727,5 @@ INSTANTIATE_TEST_SUITE_P( VeloxShuffleWriteParam, RangePartitioningShuffleWriter, ::testing::ValuesIn(kShuffleWriteTestParams)); + } // namespace gluten diff --git a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc index 68e79c80f5b9..a1cd59c6c05a 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -27,7 +27,6 @@ #include "substrait/SubstraitToVeloxPlan.h" #include "substrait/VeloxToSubstraitPlan.h" -#include "velox/functions/sparksql/Register.h" #include "velox/vector/tests/utils/VectorTestBase.h" #include "substrait/VariantToVectorConverter.h" @@ -38,6 +37,7 @@ using namespace facebook::velox::test; using namespace facebook::velox::exec::test; namespace gluten { + class VeloxSubstraitRoundTripTest : public OperatorTestBase { protected: /// Makes a vector of INTEGER type with 'size' RowVectorPtr. @@ -367,13 +367,13 @@ TEST_F(VeloxSubstraitRoundTripTest, notNullLiteral) { .addNode([&](std::string id, core::PlanNodePtr input) { std::vector projectNames = {"a", "b", "c", "d", "e", "f", "g", "h"}; std::vector projectExpressions = { - makeConstantExpr(BOOLEAN(), (bool)1), - makeConstantExpr(TINYINT(), (int8_t)23), - makeConstantExpr(SMALLINT(), (int16_t)45), - makeConstantExpr(INTEGER(), (int32_t)678), - makeConstantExpr(BIGINT(), (int64_t)910), - makeConstantExpr(REAL(), (float)1.23), - makeConstantExpr(DOUBLE(), (double)4.56), + makeConstantExpr(BOOLEAN(), static_cast(1)), + makeConstantExpr(TINYINT(), static_cast(23)), + makeConstantExpr(SMALLINT(), static_cast(45)), + makeConstantExpr(INTEGER(), static_cast(678)), + makeConstantExpr(BIGINT(), static_cast(910)), + makeConstantExpr(REAL(), static_cast(1.23)), + makeConstantExpr(DOUBLE(), static_cast(4.56)), makeConstantExpr(VARCHAR(), "789")}; return std::make_shared( id, std::move(projectNames), std::move(projectExpressions), input); diff --git a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc index bc9e9df05c05..ea8ff790b26b 100644 --- a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc +++ b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc @@ -61,4 +61,5 @@ TEST_F(VeloxToSubstraitTypeTest, basic) { testTypeConversion(ROW({"a", "b", "c"}, {BIGINT(), ROW({"x", "y"}, {BOOLEAN(), VARCHAR()}), REAL()})); testTypeConversion(ROW({}, {})); } + } // namespace gluten diff --git a/cpp/velox/udf/Udaf.h b/cpp/velox/udf/Udaf.h index 4555bdfdf8a3..f421ecbc31e8 100644 --- a/cpp/velox/udf/Udaf.h +++ b/cpp/velox/udf/Udaf.h @@ -39,4 +39,5 @@ struct UdafEntry { #define GLUTEN_REGISTER_UDAF registerUdf #define DEFINE_REGISTER_UDAF extern "C" void GLUTEN_REGISTER_UDAF() + } // namespace gluten diff --git a/cpp/velox/udf/UdfLoader.h b/cpp/velox/udf/UdfLoader.h index 51264e67cc4d..8a28c2a35145 100644 --- a/cpp/velox/udf/UdfLoader.h +++ b/cpp/velox/udf/UdfLoader.h @@ -94,4 +94,5 @@ class UdfLoader { std::unordered_set> signatures_; std::unordered_set names_; }; + } // namespace gluten diff --git a/cpp/velox/udf/examples/MyUDAF.cc b/cpp/velox/udf/examples/MyUDAF.cc index 516404b55c3f..816880d2e71b 100644 --- a/cpp/velox/udf/examples/MyUDAF.cc +++ b/cpp/velox/udf/examples/MyUDAF.cc @@ -173,6 +173,7 @@ class MyAvgRegisterer final : public gluten::UdafRegisterer { const char* myAvgIntermediateType_ = "struct"; }; + } // namespace myavg std::vector>& globalRegisters() { diff --git a/cpp/velox/udf/examples/MyUDF.cc b/cpp/velox/udf/examples/MyUDF.cc index 783699614fbd..260629fdf946 100644 --- a/cpp/velox/udf/examples/MyUDF.cc +++ b/cpp/velox/udf/examples/MyUDF.cc @@ -33,6 +33,7 @@ static const char* kDate = "date"; static const char* kVarChar = "varchar"; namespace hivestringstring { + template struct HiveStringStringFunction { VELOX_DEFINE_FUNCTION_TYPES(T); @@ -67,6 +68,7 @@ class HiveStringStringRegisterer final : public gluten::UdfRegisterer { const std::string name_ = "org.apache.spark.sql.hive.execution.UDFStringString"; const char* arg_[2] = {kVarChar, kVarChar}; }; + } // namespace hivestringstring std::vector>& globalRegisters() { diff --git a/cpp/velox/utils/ConfigExtractor.cc b/cpp/velox/utils/ConfigExtractor.cc index a83045b03e5e..78366b37568b 100644 --- a/cpp/velox/utils/ConfigExtractor.cc +++ b/cpp/velox/utils/ConfigExtractor.cc @@ -22,6 +22,7 @@ #include "utils/Exception.h" #include "velox/connectors/hive/HiveConfig.h" +#include "velox/connectors/hive/storage_adapters/s3fs/S3Config.h" namespace { @@ -98,27 +99,39 @@ std::shared_ptr getHiveConfig( } if (useInstanceCredentials) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3UseInstanceCredentials] = "true"; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kUseInstanceCredentials)] = "true"; } else if (!iamRole.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3IamRole] = iamRole; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kIamRole)] = iamRole; if (!iamRoleSessionName.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3IamRoleSessionName] = iamRoleSessionName; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kIamRoleSessionName)] = iamRoleSessionName; } } else { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3AwsAccessKey] = awsAccessKey; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3AwsSecretKey] = awsSecretKey; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kAccessKey)] = awsAccessKey; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kSecretKey)] = awsSecretKey; } // Only need to set s3 endpoint when not use instance credentials. if (!useInstanceCredentials) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3Endpoint] = awsEndpoint; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kEndpoint)] = awsEndpoint; } - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3SSLEnabled] = sslEnabled ? "true" : "false"; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3PathStyleAccess] = pathStyleAccess ? "true" : "false"; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3LogLevel] = awsSdkLogLevel; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3MaxAttempts] = retryMaxAttempts; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3RetryMode] = retryMode; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3MaxConnections] = maxConnections; - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kS3ConnectTimeout] = connectTimeout; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kSSLEnabled)] = sslEnabled ? "true" : "false"; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kPathStyleAccess)] = pathStyleAccess ? "true" : "false"; + hiveConfMap[facebook::velox::filesystems::S3Config::kS3LogLevel] = awsSdkLogLevel; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kMaxAttempts)] = retryMaxAttempts; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kRetryMode)] = retryMode; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kMaxConnections)] = maxConnections; + hiveConfMap[facebook::velox::filesystems::S3Config::baseConfigKey( + facebook::velox::filesystems::S3Config::Keys::kConnectTimeout)] = connectTimeout; #endif #ifdef ENABLE_GCS @@ -128,7 +141,7 @@ std::shared_ptr getHiveConfig( std::string gcsEndpoint = gsStorageRootUrl.value(); if (!gcsEndpoint.empty()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSEndpoint] = gcsEndpoint; + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsEndpoint] = gcsEndpoint; } } @@ -136,13 +149,13 @@ std::shared_ptr getHiveConfig( // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedErrorCountRetryPolicy auto gsMaxRetryCount = conf->get("spark.hadoop.fs.gs.http.max.retry"); if (gsMaxRetryCount.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryCount] = gsMaxRetryCount.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryCount] = gsMaxRetryCount.value(); } // https://cloud.google.com/cpp/docs/reference/storage/latest/classgoogle_1_1cloud_1_1storage_1_1LimitedTimeRetryPolicy auto gsMaxRetryTime = conf->get("spark.hadoop.fs.gs.http.max.retry-time"); if (gsMaxRetryTime.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSMaxRetryTime] = gsMaxRetryTime.value(); + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsMaxRetryTime] = gsMaxRetryTime.value(); } // https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/CONFIGURATION.md#authentication @@ -153,7 +166,7 @@ std::shared_ptr getHiveConfig( auto gsAuthServiceAccountJsonKeyfile = conf->get("spark.hadoop.fs.gs.auth.service.account.json.keyfile"); if (gsAuthServiceAccountJsonKeyfile.hasValue()) { - hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGCSCredentialsPath] = + hiveConfMap[facebook::velox::connector::hive::HiveConfig::kGcsCredentialsPath] = gsAuthServiceAccountJsonKeyfile.value(); } else { LOG(WARNING) << "STARTUP: conf spark.hadoop.fs.gs.auth.type is set to SERVICE_ACCOUNT_JSON_KEYFILE, " diff --git a/cpp/velox/utils/HdfsUtils.cc b/cpp/velox/utils/HdfsUtils.cc index ec395f817cdf..f03ca9c17dbb 100644 --- a/cpp/velox/utils/HdfsUtils.cc +++ b/cpp/velox/utils/HdfsUtils.cc @@ -23,6 +23,7 @@ namespace gluten { namespace { + struct Credential { const std::string userName; const std::string allTokens; @@ -34,6 +35,7 @@ struct Credential { return !(rhs == *this); } }; + } // namespace void updateHdfsTokens(const facebook::velox::config::ConfigBase* veloxCfg) { @@ -63,4 +65,5 @@ void updateHdfsTokens(const facebook::velox::config::ConfigBase* veloxCfg) { hdfsSetTokenForDefaultUser(token.data()); activeCredential.emplace(newCredential); } + } // namespace gluten diff --git a/cpp/velox/utils/VeloxArrowUtils.cc b/cpp/velox/utils/VeloxArrowUtils.cc index 0349eb718b8e..f26b49a4768d 100644 --- a/cpp/velox/utils/VeloxArrowUtils.cc +++ b/cpp/velox/utils/VeloxArrowUtils.cc @@ -16,6 +16,9 @@ */ #include "utils/VeloxArrowUtils.h" + +#include + #include "memory/VeloxColumnarBatch.h" #include "utils/Common.h" #include "velox/vector/ComplexVector.h" diff --git a/cpp/velox/utils/VeloxBatchResizer.cc b/cpp/velox/utils/VeloxBatchResizer.cc index 56429299464a..851af9c83cd9 100644 --- a/cpp/velox/utils/VeloxBatchResizer.cc +++ b/cpp/velox/utils/VeloxBatchResizer.cc @@ -18,8 +18,8 @@ #include "VeloxBatchResizer.h" namespace gluten { - namespace { + class SliceRowVector : public ColumnarBatchIterator { public: SliceRowVector(int32_t maxOutputBatchSize, facebook::velox::RowVectorPtr in) @@ -115,4 +115,5 @@ std::shared_ptr VeloxBatchResizer::next() { int64_t VeloxBatchResizer::spillFixedSize(int64_t size) { return in_->spillFixedSize(size); } + } // namespace gluten diff --git a/cpp/velox/utils/VeloxBatchResizer.h b/cpp/velox/utils/VeloxBatchResizer.h index 522baf4982e4..d3d1539dd7f8 100644 --- a/cpp/velox/utils/VeloxBatchResizer.h +++ b/cpp/velox/utils/VeloxBatchResizer.h @@ -22,6 +22,7 @@ #include "velox/vector/ComplexVector.h" namespace gluten { + class VeloxBatchResizer : public ColumnarBatchIterator { public: VeloxBatchResizer( @@ -42,4 +43,5 @@ class VeloxBatchResizer : public ColumnarBatchIterator { std::unique_ptr next_ = nullptr; }; + } // namespace gluten diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.cc b/cpp/velox/utils/tests/MemoryPoolUtils.cc index 5a0ae03b1496..2d4e19511a6e 100644 --- a/cpp/velox/utils/tests/MemoryPoolUtils.cc +++ b/cpp/velox/utils/tests/MemoryPoolUtils.cc @@ -152,4 +152,5 @@ arrow::Status SelfEvictedMemoryPool::ensureCapacity(int64_t size) { } return arrow::Status::OK(); } -} // namespace gluten \ No newline at end of file + +} // namespace gluten diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.h b/cpp/velox/utils/tests/MemoryPoolUtils.h index 68e279008497..5fdf880be678 100644 --- a/cpp/velox/utils/tests/MemoryPoolUtils.h +++ b/cpp/velox/utils/tests/MemoryPoolUtils.h @@ -24,9 +24,7 @@ namespace gluten { -/** - * arrow::MemoryPool instance with limited capacity, used by tests and benchmarks - */ +// arrow::MemoryPool instance with limited capacity, used by tests and benchmarks. class LimitedMemoryPool final : public arrow::MemoryPool { public: explicit LimitedMemoryPool() : capacity_(std::numeric_limits::max()) {} @@ -54,9 +52,7 @@ class LimitedMemoryPool final : public arrow::MemoryPool { arrow::internal::MemoryPoolStats stats_; }; -/** - * arrow::MemoryPool instance with limited capacity and can be evictable on OOM, used by tests and benchmarks - */ +// arrow::MemoryPool instance with limited capacity and can be evictable on OOM, used by tests and benchmarks. class SelfEvictedMemoryPool : public arrow::MemoryPool { public: explicit SelfEvictedMemoryPool(arrow::MemoryPool* pool, bool failIfOOM = true) : pool_(pool), failIfOOM_(failIfOOM) {} diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h index f5dd6f4f384a..d2995e251c68 100644 --- a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -90,6 +90,10 @@ class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase protected: void setUp() { + if (!isRegisteredNamedVectorSerde(facebook::velox::VectorSerde::Kind::kPresto)) { + // RSS shuffle serde. + facebook::velox::serializer::presto::PrestoVectorSerde::registerNamedVectorSerde(); + } // Set up test data. children1_ = { makeNullableFlatVector({1, 2, 3, std::nullopt, 4, std::nullopt, 5, 6, std::nullopt, 7}), @@ -362,6 +366,7 @@ class VeloxShuffleWriterTest : public ::testing::TestWithParam::max(), + kDefaultReadBufferSize, defaultArrowMemoryPool().get(), pool_, GetParam().shuffleWriterType); diff --git a/dev/build-thirdparty.sh b/dev/build-thirdparty.sh index 90fc5e58fbd5..ee827ef197f7 100755 --- a/dev/build-thirdparty.sh +++ b/dev/build-thirdparty.sh @@ -82,4 +82,4 @@ elif [ "$LINUX_OS" == "debian" ]; then fi fi cd $THIRDPARTY_LIB/ -jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ +$JAVA_HOME/bin/jar cvf gluten-thirdparty-lib-$LINUX_OS-$VERSION-$ARCH.jar ./ diff --git a/dev/ci-velox-buildshared-centos-8.sh b/dev/ci-velox-buildshared-centos-8.sh index f295632ffe35..ff9a62b798b5 100755 --- a/dev/ci-velox-buildshared-centos-8.sh +++ b/dev/ci-velox-buildshared-centos-8.sh @@ -2,8 +2,6 @@ set -e -# TODO: will be removed after docker is updated. -dnf install -y --setopt=install_weak_deps=False gcc-toolset-11 source /opt/rh/gcc-toolset-11/enable ./dev/builddeps-veloxbe.sh --run_setup_script=OFF --build_arrow=OFF --enable_ep_cache=OFF --build_tests=ON \ --build_examples=ON --build_benchmarks=ON diff --git a/dev/ci-velox-buildstatic-centos-7.sh b/dev/ci-velox-buildstatic-centos-7.sh index 76bd33d7ffa7..6895891a98e9 100755 --- a/dev/ci-velox-buildstatic-centos-7.sh +++ b/dev/ci-velox-buildstatic-centos-7.sh @@ -2,8 +2,6 @@ set -e -# TODO: will be removed after docker is updated. -sudo yum install -y devtoolset-11 source /opt/rh/devtoolset-11/enable export NUM_THREADS=4 ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --build_arrow=OFF --build_tests=OFF --build_benchmarks=OFF \ diff --git a/dev/docker/Dockerfile.centos7-static-build b/dev/docker/Dockerfile.centos7-static-build index a0da8bda5b47..6de2c73cd7b7 100644 --- a/dev/docker/Dockerfile.centos7-static-build +++ b/dev/docker/Dockerfile.centos7-static-build @@ -3,7 +3,7 @@ FROM centos:7 RUN sed -i -e "s|mirrorlist=|#mirrorlist=|g" /etc/yum.repos.d/CentOS-* || true RUN sed -i -e "s|#baseurl=http://mirror.centos.org|baseurl=http://vault.centos.org|g" /etc/yum.repos.d/CentOS-* || true -RUN yum install -y centos-release-scl +RUN yum install -y epel-release centos-release-scl RUN rm /etc/yum.repos.d/CentOS-SCLo-scl.repo -f RUN sed -i \ -e 's/^mirrorlist/#mirrorlist/' \ diff --git a/dev/vcpkg/README.md b/dev/vcpkg/README.md index 229e11cc7204..986a8ec689eb 100644 --- a/dev/vcpkg/README.md +++ b/dev/vcpkg/README.md @@ -15,7 +15,7 @@ Please install build depends on your system to compile all libraries: sudo $GLUTEN_REPO/dev/vcpkg/setup-build-depends.sh ``` -For CentOS user, gcc 11 needs to be enabled manually before next step: +GCC-11 is the minimum required compiler. It needs to be enabled beforehand. Take Centos-7/8 as example: ``` sh # CentOS 8 diff --git a/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch b/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch deleted file mode 100644 index 7799dfb9e80e..000000000000 --- a/dev/vcpkg/ports/jemalloc/fix-configure-ac.patch +++ /dev/null @@ -1,13 +0,0 @@ -diff --git a/configure.ac b/configure.ac -index f6d25f334..3115504e2 100644 ---- a/configure.ac -+++ b/configure.ac -@@ -1592,7 +1592,7 @@ fi - [enable_uaf_detection="0"] - ) - if test "x$enable_uaf_detection" = "x1" ; then -- AC_DEFINE([JEMALLOC_UAF_DETECTION], [ ]) -+ AC_DEFINE([JEMALLOC_UAF_DETECTION], [ ], ["enable UAF"]) - fi - AC_SUBST([enable_uaf_detection]) - diff --git a/dev/vcpkg/ports/jemalloc/portfile.cmake b/dev/vcpkg/ports/jemalloc/portfile.cmake deleted file mode 100644 index 6cac12ca3b7c..000000000000 --- a/dev/vcpkg/ports/jemalloc/portfile.cmake +++ /dev/null @@ -1,79 +0,0 @@ -vcpkg_from_github( - OUT_SOURCE_PATH SOURCE_PATH - REPO jemalloc/jemalloc - REF 54eaed1d8b56b1aa528be3bdd1877e59c56fa90c - SHA512 527bfbf5db9a5c2b7b04df4785b6ae9d445cff8cb17298bf3e550c88890d2bd7953642d8efaa417580610508279b527d3a3b9e227d17394fd2013c88cb7ae75a - HEAD_REF master - PATCHES - fix-configure-ac.patch - preprocessor.patch -) -if(VCPKG_TARGET_IS_WINDOWS) - set(opts "ac_cv_search_log=none required" - "--without-private-namespace" - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" - "--disable-cxx" - "--disable-libdl" - # For fixing an issue when loading native lib: cannot allocate memory in static TLS block. - "--disable-initial-exec-tls" - "CFLAGS=-fPIC" - "CXXFLAGS=-fPIC") -else() - set(opts - "--with-jemalloc-prefix=je_gluten_" - "--with-private-namespace=je_gluten_private_" - "--without-export" - "--disable-shared" - "--disable-cxx" - "--disable-libdl" - # For fixing an issue when loading native lib: cannot allocate memory in static TLS block. - "--disable-initial-exec-tls" - "CFLAGS=-fPIC" - "CXXFLAGS=-fPIC") -endif() - -vcpkg_configure_make( - SOURCE_PATH "${SOURCE_PATH}" - AUTOCONFIG - NO_WRAPPERS - OPTIONS ${opts} -) - -vcpkg_install_make() - -if(VCPKG_TARGET_IS_WINDOWS) - file(COPY "${SOURCE_PATH}/include/msvc_compat/strings.h" DESTINATION "${CURRENT_PACKAGES_DIR}/include/jemalloc/msvc_compat") - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/include/jemalloc/jemalloc.h" "" "\"msvc_compat/strings.h\"") - if(VCPKG_LIBRARY_LINKAGE STREQUAL "dynamic") - file(COPY "${CURRENT_BUILDTREES_DIR}/${TARGET_TRIPLET}-rel/lib/jemalloc.lib" DESTINATION "${CURRENT_PACKAGES_DIR}/lib") - file(MAKE_DIRECTORY "${CURRENT_PACKAGES_DIR}/bin") - file(RENAME "${CURRENT_PACKAGES_DIR}/lib/jemalloc.dll" "${CURRENT_PACKAGES_DIR}/bin/jemalloc.dll") - endif() - if(NOT VCPKG_BUILD_TYPE) - if(VCPKG_LIBRARY_LINKAGE STREQUAL "dynamic") - file(COPY "${CURRENT_BUILDTREES_DIR}/${TARGET_TRIPLET}-dbg/lib/jemalloc.lib" DESTINATION "${CURRENT_PACKAGES_DIR}/debug/lib") - file(MAKE_DIRECTORY "${CURRENT_PACKAGES_DIR}/debug/bin") - file(RENAME "${CURRENT_PACKAGES_DIR}/debug/lib/jemalloc.dll" "${CURRENT_PACKAGES_DIR}/debug/bin/jemalloc.dll") - endif() - endif() - if(VCPKG_LIBRARY_LINKAGE STREQUAL "static") - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/lib/pkgconfig/jemalloc.pc" "install_suffix=" "install_suffix=_s") - if(NOT VCPKG_BUILD_TYPE) - vcpkg_replace_string("${CURRENT_PACKAGES_DIR}/debug/lib/pkgconfig/jemalloc.pc" "install_suffix=" "install_suffix=_s") - endif() - endif() -endif() - -vcpkg_fixup_pkgconfig() - -vcpkg_copy_pdbs() - -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/debug/include") -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/debug/share") -file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/tools") - -# Handle copyright -file(INSTALL "${SOURCE_PATH}/COPYING" DESTINATION "${CURRENT_PACKAGES_DIR}/share/${PORT}" RENAME copyright) diff --git a/dev/vcpkg/ports/jemalloc/preprocessor.patch b/dev/vcpkg/ports/jemalloc/preprocessor.patch deleted file mode 100644 index 6e6e2d1403fb..000000000000 --- a/dev/vcpkg/ports/jemalloc/preprocessor.patch +++ /dev/null @@ -1,12 +0,0 @@ -diff --git a/configure.ac b/configure.ac -index 3115504e2..ffb504b08 100644 ---- a/configure.ac -+++ b/configure.ac -@@ -749,6 +749,7 @@ case "${host}" in - so="dll" - if test "x$je_cv_msvc" = "xyes" ; then - importlib="lib" -+ JE_APPEND_VS(CPPFLAGS, -DJEMALLOC_NO_PRIVATE_NAMESPACE) - DSO_LDFLAGS="-LD" - EXTRA_LDFLAGS="-link -DEBUG" - CTARGET='-Fo$@' diff --git a/dev/vcpkg/ports/jemalloc/vcpkg.json b/dev/vcpkg/ports/jemalloc/vcpkg.json deleted file mode 100644 index 007e05b931c9..000000000000 --- a/dev/vcpkg/ports/jemalloc/vcpkg.json +++ /dev/null @@ -1,8 +0,0 @@ -{ - "name": "jemalloc", - "version": "5.3.0", - "port-version": 1, - "description": "jemalloc is a general purpose malloc(3) implementation that emphasizes fragmentation avoidance and scalable concurrency support", - "homepage": "https://jemalloc.net/", - "license": "BSD-2-Clause" -} diff --git a/dev/vcpkg/ports/libelf/portfile.cmake b/dev/vcpkg/ports/libelf/portfile.cmake index 8e9ea0c71fe1..d1b347d69352 100644 --- a/dev/vcpkg/ports/libelf/portfile.cmake +++ b/dev/vcpkg/ports/libelf/portfile.cmake @@ -1,5 +1,5 @@ vcpkg_download_distfile(ARCHIVE - URLS "https://ftp.osuosl.org/pub/blfs/conglomeration/libelf/libelf-0.8.13.tar.gz" + URLS "https://fossies.org/linux/misc/old/libelf-0.8.13.tar.gz" FILENAME "libelf-0.8.13.tar.gz" SHA512 d2a4ea8ccc0bbfecac38fa20fbd96aefa8e86f8af38691fb6991cd9c5a03f587475ecc2365fc89a4954c11a679d93460ee9a5890693112f6133719af3e6582fe ) diff --git a/dev/vcpkg/vcpkg.json b/dev/vcpkg/vcpkg.json index b3d8dc2fcc0c..c0123cfbe998 100644 --- a/dev/vcpkg/vcpkg.json +++ b/dev/vcpkg/vcpkg.json @@ -1,7 +1,6 @@ { "$schema": "https://raw.githubusercontent.com/microsoft/vcpkg-tool/main/docs/vcpkg.schema.json", "builtin-baseline": "a7b6122f6b6504d16d96117336a0562693579933", - "dependencies": ["jemalloc"], "default-features": ["velox"], "features": { "velox": { @@ -50,7 +49,6 @@ "xxhash", "protobuf", "benchmark", - "jemalloc", "icu", "thrift", "libstemmer" @@ -92,7 +90,8 @@ "description": "Velox ABFS Support", "dependencies": [ "azure-storage-blobs-cpp", - "azure-storage-files-datalake-cpp" + "azure-storage-files-datalake-cpp", + "azure-identity-cpp" ] }, "duckdb": { diff --git a/docs/Configuration.md b/docs/Configuration.md index 76549dd4fe7c..e217be45ff80 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -23,7 +23,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.shuffle.manager | To turn on Gluten Columnar Shuffle Plugin | org.apache.spark.shuffle.sort.ColumnarShuffleManager | | spark.gluten.enabled | Enable Gluten, default is true. Just an experimental property. Recommend to enable/disable Gluten through the setting for `spark.plugins`. | true | | spark.gluten.memory.isolation | (Experimental) Enable isolated memory mode. If true, Gluten controls the maximum off-heap memory can be used by each task to X, X = executor memory / max task slots. It's recommended to set true if Gluten serves concurrent queries within a single session, since not all memory Gluten allocated is guaranteed to be spillable. In the case, the feature should be enabled to avoid OOM. Note when true, setting spark.memory.storageFraction to a lower value is suggested since storage memory is considered non-usable by Gluten. | false | -| spark.gluten.ras.enabled | Experimental: Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature is still in development and may not bring performance profits. | false | +| spark.gluten.ras.enabled | Enables RAS (relation algebra selector) during physical planning to generate more efficient query plan. Note, this feature doesn't bring performance profits by default. Try exploring option `spark.gluten.ras.costModel` for advanced usage. | false | | spark.gluten.sql.columnar.maxBatchSize | Number of rows to be processed in each batch. Default value is 4096. | 4096 | | spark.gluten.sql.columnar.scanOnly | When enabled, this config will overwrite all other operators' enabling, and only Scan and Filter pushdown will be offloaded to native. | false | | spark.gluten.sql.columnar.batchscan | Enable or Disable Columnar BatchScan, default is true | true | @@ -51,6 +51,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.shuffle.compression.threshold | If number of rows in a batch falls below this threshold, will copy all buffers into one buffer to compress. | 100 | | spark.gluten.sql.columnar.shuffle.realloc.threshold | Set the threshold to dynamically adjust the size of shuffle split buffers. The size of each split buffer is recalculated for each incoming batch of data. If the new size deviates from the current partition buffer size by a factor outside the range of [1 - threshold, 1 + threshold], the split buffer will be re-allocated using the newly calculated size | 0.25 | | spark.gluten.sql.columnar.shuffle.merge.threshold | Set the threshold control the minimum merged size. When a partition buffer is full, and the number of rows is below (`threshold * spark.gluten.sql.columnar.maxBatchSize`), it will be saved for merging. | 0.25 | +| spark.gluten.sql.columnar.shuffle.readerBufferSize | Buffer size in bytes for shuffle reader reading input stream from local or remote. | 1MB | | spark.gluten.sql.columnar.numaBinding | Set up NUMABinding, default is false | true | | spark.gluten.sql.columnar.coreRange | Set up the core range for NUMABinding, only works when numaBinding set to true.
The setting is based on the number of cores in your system. Use 72 cores as an example. | 0-17,36-53 |18-35,54-71 | | spark.gluten.sql.columnar.wholeStage.fallback.threshold | Configure the threshold for whether whole stage will fall back in AQE supported case by counting the number of ColumnarToRow & vanilla leaf node | \>= 1 | @@ -58,6 +59,7 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.fallback.ignoreRowToColumnar | When true, the fallback policy ignores the RowToColumnar when counting fallback number. | true | | spark.gluten.sql.columnar.fallback.preferColumnar | When true, the fallback policy prefers to use Gluten plan rather than vanilla Spark plan if the both of them contains ColumnarToRow and the vanilla Spark plan ColumnarToRow number is not smaller than Gluten plan. | true | | spark.gluten.sql.columnar.force.hashagg | Force to use hash agg to replace sort agg. | true | +| spark.gluten.sql.mergeTwoPhasesAggregate.enabled | Whether to merge two phases aggregate if there are no other operators between them. | true | | spark.gluten.sql.columnar.vanillaReaders | Enable vanilla spark's vectorized reader. Please note it may bring perf. overhead due to extra data transition. We recommend to disable it if most queries can be fully offloaded to gluten. | false | | spark.gluten.sql.native.bloomFilter | Enable or Disable native runtime bloom filter. | true | | spark.gluten.sql.native.arrow.reader.enabled | Enable or Disable native arrow read CSV file format | false | @@ -87,7 +89,7 @@ The following configurations are related to Velox settings. | spark.gluten.sql.columnar.backend.velox.filePreloadThreshold | Set the file preload threshold for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.prefetchRowGroups | Set the prefetch row groups for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.loadQuantum | Set the load quantum for velox file scan. | | -| spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes | Set the max coalesced distance bytes for velox file scan. | | +| spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance | Set the max coalesced distance for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes | Set the max coalesced bytes for velox file scan. | | | spark.gluten.sql.columnar.backend.velox.cachePrefetchMinPct | Set prefetch cache min pct for velox file scan. | | | spark.gluten.velox.awsSdkLogLevel | Log granularity of AWS C++ SDK in velox. | FATAL | diff --git a/docs/developers/CppCodingStyle.md b/docs/developers/CppCodingStyle.md index 42101882a9e5..2cea8fe8feb0 100644 --- a/docs/developers/CppCodingStyle.md +++ b/docs/developers/CppCodingStyle.md @@ -73,7 +73,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * Obvious exception: files used for defining various misc functions. * If a header file has a corresponding source file, they should have the same file name with different suffix, such as `a.h vs a.cc`. -* If a function is declared in the file `a.h`, ensure it's defined in the corrosponding +* If a function is declared in the file `a.h`, ensure it's defined in the corresponding source file `a.cc`, do not define it in other files. * No deep source directory for CPP files, not do it as JAVA. * Include header files should satisfy the following rules. @@ -95,7 +95,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * Ensure that public inheritance represent the relation of `is-a`. * Ensure that private inheritance represent the relation of `implements-with`. * Don't make a function `virtual` without reason. -* Ensure the polymorphic base class has a `virtual` deconstructor. +* Ensure the polymorphic base class has a `virtual` destructor. * Use `override` to make overriding explicit and to make the compiler work. * Use `const` to mark the member function read-only as far as possible. * When you try to define a `copy constructor` or a `operator=` for a class, remember @@ -113,8 +113,8 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt the function name. such as. use `get(size_t index)` instead of `getByIndex`. * A function should focus on a single logic operation. * A function should do as the name meaning. - * do everything converd by the function name - * don't do anything not convered by the function name + * do everything covered by the function name + * don't do anything not covered by the function name ## Variable @@ -158,7 +158,7 @@ cmake-format --first-comment-is-literal True --in-place cpp/velox/CMakeLists.txt * `unique_ptr` represents ownership, but not share ownership. `unique_ptr` is equivalent to RAII, release the resource when the object is destructed. * `shared_ptr` represents shared ownership by use-count. It is more expensive - that `unqiue_ptr`. + that `unique_ptr`. * `weak_ptr` models temporary ownership. It is useful in breaking reference cycles formed by objects managed by `shared_ptr`. * Use `unique_ptr` or `shared_ptr` to represent ownership. diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md index c4b1a03a36d8..22ad3e30efc7 100644 --- a/docs/developers/HowTo.md +++ b/docs/developers/HowTo.md @@ -122,6 +122,13 @@ gdb ${GLUTEN_HOME}/cpp/build/releases/libgluten.so 'core-Executor task l-2000883 ``` - the `core-Executor task l-2000883-1671542526` represents the core file name. +# How to use jemalloc for Gluten native engine + +Currently, we have no dedicated memory allocator implemented by jemalloc. User can set environment variable `LD_PRELOAD` for lib jemalloc +to let it override the corresponding C standard functions entirely. It may help alleviate OOM issues. + +`spark.executorEnv.LD_PREALOD=/path/to/libjemalloc.so` + # How to run TPC-H on Velox backend Now, both Parquet and DWRF format files are supported, related scripts and files are under the directory of `${GLUTEN_HOME}/backends-velox/workload/tpch`. diff --git a/docs/developers/MicroBenchmarks.md b/docs/developers/MicroBenchmarks.md index bd469f34c81c..c59d6be3a631 100644 --- a/docs/developers/MicroBenchmarks.md +++ b/docs/developers/MicroBenchmarks.md @@ -15,7 +15,7 @@ comparing with directly debugging in a Spark job. Developers can use it to creat workloads, debug in native process, profile the hotspot and do optimizations. To simulate a first stage, you need to dump the Substrait plan and input split info into two JSON -files. The input URIs of the splits should be exising file locations, which can be either local or +files. The input URIs of the splits should be existing file locations, which can be either local or HDFS paths. To simulate a middle stage, in addition to the JSON file, you also need to save the input data of @@ -64,7 +64,7 @@ cd /path/to/gluten/cpp/build/velox/benchmarks --plan /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example.json \ --data /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example_orders/part-00000-1e66fb98-4dd6-47a6-8679-8625dbc437ee-c000.snappy.parquet,\ /home/sparkuser/github/apache/incubator-gluten/backends-velox/generated-native-benchmark/example_lineitem/part-00000-3ec19189-d20e-4240-85ae-88631d46b612-c000.snappy.parquet \ ---threads 1 --iterations 1 --noprint-result --benchmark_filter=InputFromBatchStream +--threads 1 --iterations 1 --noprint-result ``` The output should be like: @@ -118,12 +118,12 @@ cd /path/to/gluten/ First, get the Stage Id from spark UI for the stage you want to simulate. And then re-run the query with below configurations to dump the inputs to micro benchmark. -| Parameters | Description | Recommend Setting | -|---------------------------------------------|----------------------------------------------------------------------------------------------------------------|-----------------------| -| spark.gluten.sql.benchmark_task.stageId | Spark task stage id | target stage id | -| spark.gluten.sql.benchmark_task.partitionId | Spark task partition id, default value -1 means all the partition of this stage | 0 | -| spark.gluten.sql.benchmark_task.taskId | If not specify partition id, use spark task attempt id, default value -1 means all the partition of this stage | target task attemp id | -| spark.gluten.saveDir | Directory to save the inputs to micro benchmark, should exist and be empty. | /path/to/saveDir | +| Parameters | Description | Recommend Setting | +|---------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------| +| spark.gluten.sql.benchmark_task.taskId | Comma-separated string to specify the Task IDs to dump. If it's set, `spark.gluten.sql.benchmark_task.stageId` and `spark.gluten.sql.benchmark_task.partitionId` will be ignored. | Comma-separated string of task IDs. Empty by default. | +| spark.gluten.sql.benchmark_task.stageId | Spark stage ID. | Target stage ID | +| spark.gluten.sql.benchmark_task.partitionId | Comma-separated string to specify the Partition IDs in a stage to dump. Must be specified together with `spark.gluten.sql.benchmark_task.stageId`. Empty by default, meaning all partitions of this stage will be dumped. To identify the partition ID, navigate to the `Stage` tab in the Spark UI and locate it under the `Index` column. | Comma-separated string of partition IDs. Empty by default. | +| spark.gluten.saveDir | Directory to save the inputs to micro benchmark, should exist and be empty. | /path/to/saveDir | Check the files in `spark.gluten.saveDir`. If the simulated stage is a first stage, you will get 3 or 4 types of dumped file: @@ -257,7 +257,7 @@ inputs from a first stage. The steps are demonstrated as below: 1. Start spark-shell or pyspark We need to set `spark.gluten.sql.benchmark_task.stageId` and `spark.gluten.saveDir` to dump the inputs. -Normally, the stage id should be greater than 0. You can run the command in step 2 in advance to get the +Normally, the stage id should be greater than 0. You can run the command in step 2 in advance to get the right stage id in your case. We shall set `spark.default.parallelism` to 1 and `spark.sql.files.maxPartitionBytes` large enough to make sure there will be only 1 task in the first stage. @@ -358,6 +358,11 @@ ShuffleWriteRead/iterations:1/process_time/real_time/threads:1 121637629714 ns Unless `spark.gluten.sql.debug` is set in the INI file via `--conf`, the logging behavior is same as debug mode off. Developers can use `--debug-mode` command line flag to turn on debug mode when needed, and set verbosity/severity level via command line flags `--v` and `--minloglevel`. Note that constructing and deconstructing log strings can be very time-consuming, which may cause benchmark times to be inaccurate. + +## Enable HDFS support + +After enabling the dynamic loading of libhdfs.so at runtime to support HDFS, if you run the benchmark with an HDFS file, you need to set the classpath for Hadoop. You can do this by running `export CLASSPATH=`$HADOOP_HOME/bin/hdfs classpath --glob``. Otherwise, the HDFS connection will fail. If you have replaced ${HADOOP_HOME}/lib/native/libhdfs.so with libhdfs3.so, there is no need to set the `CLASSPATH`. + ## Simulate write tasks The last operator for a write task is a file write operator, and the output from Velox pipeline only @@ -373,17 +378,18 @@ cd /path/to/gluten/cpp/build/velox/benchmarks --write-path /absolute_path/
``` - ## Simulate task spilling -You can simulate task spilling by specify memory hard limit from `--memory_limit`. +You can simulate task spilling by specify a memory hard limit from `--memory_limit`. By default, spilled files are written to the `/tmp` directory. +To simulate real Gluten workloads, which utilize multiple spill directories, set the environment variable GLUTEN_SPARK_LOCAL_DIRS to a comma-separated string. +Please check [Simulate Gluten workload with multiple processes and threads](#Simulate-Gluten-workload-with-multiple-processes-and-threads) for more details. -## Simulate Spark with multiple processes and threads +## Simulate Gluten workload with multiple processes and threads You can use below command to launch several processes and threads to simulate parallel execution on Spark. Each thread in the same process will be pinned to the core number starting from `--cpu`. -Suppose running on a baremetal machine with 48C, 2-socket, HT-on, launching below command will +Suppose running on a bare-metal machine with 48C, 2-socket, HT-on, launching below command will utilize all vcores. ```shell @@ -395,9 +401,10 @@ for ((i=0; i<${processes}; i++)); do done ``` -If you want to add the shuffle write process, you can specify multiple directories by setting -environment variable `GLUTEN_SPARK_LOCAL_DIRS` to a comma-separated string for shuffle write to -spread the I/O pressure to multiple disks. +To include the shuffle write process or trigger spilling via `--memory-limit`, +you can specify multiple directories by setting the `GLUTEN_SPARK_LOCAL_DIRS` environment variable +to a comma-separated string. This will distribute the I/O load across multiple disks, similar to how it works for Gluten workloads. +Temporary subdirectories will be created under each specified directory at runtime and will be automatically deleted if the process completes normally. ```shell mkdir -p {/data1,/data2,/data3}/tmp # Make sure each directory has been already created. diff --git a/docs/developers/NewToGluten.md b/docs/developers/NewToGluten.md index 8228c919e850..b790d8740a9e 100644 --- a/docs/developers/NewToGluten.md +++ b/docs/developers/NewToGluten.md @@ -55,10 +55,10 @@ spark.executor.extraJavaOptions=-Dio.netty.tryReflectionSetAccessible=true ## Maven 3.6.3 or above -[Maven Dowload Page](https://maven.apache.org/docs/history.html) +[Maven Download Page](https://maven.apache.org/docs/history.html) And then set the environment setting. -## GCC 9.4 or above +## GCC 11 or above # Compile gluten using debug mode @@ -247,7 +247,7 @@ Then you can create breakpoint and debug in `Run and Debug` section. ### Velox debug For some velox tests such as `ParquetReaderTest`, tests need to read the parquet file in `/velox/dwio/parquet/tests/examples`, -you should let the screen on `ParquetReaderTest.cpp`, then click `Start Debuging`, otherwise `No such file or directory` exception will be raised. +you should let the screen on `ParquetReaderTest.cpp`, then click `Start Debugging`, otherwise `No such file or directory` exception will be raised. ## Useful notes diff --git a/docs/developers/ProfileMemoryOfGlutenWithVelox.md b/docs/developers/ProfileMemoryOfGlutenWithVelox.md index 480755cb99c5..1f57f809218c 100644 --- a/docs/developers/ProfileMemoryOfGlutenWithVelox.md +++ b/docs/developers/ProfileMemoryOfGlutenWithVelox.md @@ -5,13 +5,14 @@ nav_order: 8 has_children: true parent: /developer-overview/ --- -Gluten offloads most of computation to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) to analyze the offheap memory and cpu profile. +Gluten offloads most of Spark SQL execution to native engine. We can use [gperftools](https://github.com/gperftools/gperftools) or [jemalloc](https://github.com/jemalloc/jemalloc) +to analyze the offheap memory and cpu profile. -# Profiling using gperftools +# Profile with gperftools `gperftools` is a collection of a high-performance multi-threaded malloc() implementation, plus some pretty nifty performance analysis -tools, see more: https://github.com/gperftools/gperftools/wiki +tools, see more: https://github.com/gperftools/gperftools/wiki. ## Build and install gperftools @@ -29,10 +30,10 @@ Then we can find the tcmalloc libraries in `$GPERFTOOLS_HOME/.lib`. ## Run Gluten with gperftools -Use `--file` or `spark.files` to upload tcmalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/gperftools/libtcmalloc_and_profiler.so +--files /path/to/gperftools/libtcmalloc_and_profiler.so or spark.files /path/to/gperftools/libtcmalloc_and_profiler.so ``` @@ -48,14 +49,14 @@ spark.executorEnv.LD_PRELOAD ./libtcmalloc_and_profiler.so spark.executorEnv.HEAPPROFILE /tmp/gluten_heap_perf_${CONTAINER_ID} ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf_${CONTAINER_ID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so ralative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` @@ -82,9 +83,11 @@ Result like: **\*\*** Get more help from https://github.com/gperftools/gperftools/wiki#documentation. -# Profiling using jemalloc +# Profile with jemalloc -`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. +`jemalloc` is a general purpose malloc(3) implementation that emphasizes fragmentation +avoidance and scalable concurrency support. We can also use it to analyze Gluten performance. +Getting Started with `jemalloc`: https://github.com/jemalloc/jemalloc/wiki/Getting-Started. ## Build and install jemalloc @@ -99,10 +102,10 @@ Then we can find the jemalloc library in `$JEMALLOC_HOME/.lib`. ## Run Gluten with jemalloc -Use `--file` or `spark.files` to upload jemalloc library. +Configure `--files` or `spark.files` for Spark. ``` ---file /path/to/jemalloc/libjemalloc.so +--files /path/to/jemalloc/libjemalloc.so or spark.files /path/to/jemalloc/libjemalloc.so ``` @@ -114,14 +117,14 @@ spark.executorEnv.LD_PRELOAD ./libjemalloc.so spark.executorEnv.MALLOC_CONF prof:true,lg_prof_interval:30,prof_prefix:/tmp/gluten_heap_perf ``` -Finally, profiling files starting with `/tmp/gluten_heap_perf.${PID}` will be generated in each spark executor. +Finally, profiling files prefixed with `/tmp/gluten_heap_perf.${PID}` will be generated for each spark executor. -## Analyze output profiles +## Analyze profiling output -Prepare the required native libraries. We can extract the gluten and velox libraries from gluten bundle jar. (Maybe also need dependency libraries for non-static compilation) +Prepare the required native libraries. Assume static build is used for Gluten, so there is no other shared dependency libs. ```bash -jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar libvelox.so libgluten.so +jar xf gluten-velox-bundle-spark3.5_2.12-centos_7_x86_64-1.2.0.jar relative/path/to/libvelox.so relative/path/to/libgluten.so mv libvelox.so libgluten.so /path/to/gluten_lib_prefix ``` diff --git a/docs/developers/VeloxUDF.md b/docs/developers/VeloxUDF.md index 4f685cc41ed7..a8b6a86aeb55 100644 --- a/docs/developers/VeloxUDF.md +++ b/docs/developers/VeloxUDF.md @@ -38,7 +38,7 @@ The following steps demonstrate how to set up a UDF library project: - `registerUdf()`: This function is called to register the UDF to Velox function registry. - This is where users should register functions by calling `facebook::velox::exec::registerVecotorFunction` or other Velox APIs. + This is where users should register functions by calling `facebook::velox::exec::registerVectorFunction` or other Velox APIs. - The interface functions are mapped to marcos in [Udf.h](../../cpp/velox/udf/Udf.h). @@ -172,22 +172,23 @@ or Start `spark-sql` and run query. You need to add jar "spark-hive_2.12--tests.jar" to the classpath for hive udf `org.apache.spark.sql.hive.execution.UDFStringString` ``` +spark-sql (default)> create table tbl as select * from values ('hello'); +Time taken: 3.656 seconds spark-sql (default)> CREATE TEMPORARY FUNCTION hive_string_string AS 'org.apache.spark.sql.hive.execution.UDFStringString'; -Time taken: 0.808 seconds -spark-sql (default)> select hive_string_string("hello", "world"); +Time taken: 0.047 seconds +spark-sql (default)> select hive_string_string(col1, 'world') from tbl; hello world -Time taken: 3.208 seconds, Fetched 1 row(s) +Time taken: 1.217 seconds, Fetched 1 row(s) ``` You can verify the offload with "explain". ``` -spark-sql (default)> explain select hive_string_string("hello", "world"); -== Physical Plan == -VeloxColumnarToRowExec -+- ^(2) ProjectExecTransformer [hello world AS hive_string_string(hello, world)#8] - +- ^(2) InputIteratorTransformer[fake_column#9] +spark-sql (default)> explain select hive_string_string(col1, 'world') from tbl; +VeloxColumnarToRow ++- ^(2) ProjectExecTransformer [HiveSimpleUDF#org.apache.spark.sql.hive.execution.UDFStringString(col1#11,world) AS hive_string_string(col1, world)#12] + +- ^(2) InputIteratorTransformer[col1#11] +- RowToVeloxColumnar - +- *(1) Scan OneRowRelation[fake_column#9] + +- Scan hive spark_catalog.default.tbl [col1#11], HiveTableRelation [`spark_catalog`.`default`.`tbl`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Data Cols: [col1#11], Partition Cols: []] ``` ## Configurations @@ -214,7 +215,7 @@ pip3 install pyspark==$SPARK_VERSION cython pip3 install pandas pyarrow ``` -Gluten provides a config to control enable `ColumnarArrowEvalPython` or not, with `true` as defalt. +Gluten provides a config to control enable `ColumnarArrowEvalPython` or not, with `true` as default. ``` spark.gluten.sql.columnar.arrowUdf diff --git a/docs/developers/clickhouse-backend-debug.md b/docs/developers/clickhouse-backend-debug.md new file mode 100644 index 000000000000..13799ac2cba2 --- /dev/null +++ b/docs/developers/clickhouse-backend-debug.md @@ -0,0 +1,98 @@ +--- +layout: page +title: Debug CH Backend +nav_order: 13 +has_children: true +parent: /developer-overview/ +--- + +# Debug CH Backend + +## Debug Java/Scala Code with IntelliJ IDEA + +1. Build Gluten ClickHouse Native Lib. + ``` + export CMAKE_BUILD_TYPE=Release && bash ep/build-clickhouse/src/build_clickhouse.sh + ``` + libch.so will be generated in `cpp-ch/build/utils/extern-local-engine/libch.so`. + +2. Maven Build Gluten ClickHouse with Profile + ``` + mvn clean install -DskipTests -Pbackends-clickhouse -Pspark-3.3 -Pspark-ut + ``` + +3. Set Maven Profiles in IntelliJ IDEA + + ![gluten-debug-profile-settings.png](../image/ClickHouse/gluten-debug-profile-settings.png) + +4. Set Debug Configuration in IntelliJ IDEA + + For example, debug GlutenMathExpressionsSuite. + ![gluten-debug-idea-config.png](../image/ClickHouse/gluten-debug-idea-config.png) + + VM Options: + `-Dtpcds.data.path=/data/tpcds-data-sf1 -Dclickhouse.lib.path=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so -Dspark.test.home=/path/to/spark33` + > Download tpcds-data in https://gluten-nginx.kyligence.com/dataset/ + > Download spark33 using `git clone --depth 1 --branch v3.3.1 https://github.com/apache/spark.git /tmp/spark33` + + Environment Variables: + `LD_PRELOAD=/path/to/gluten/cpp-ch/build/utils/extern-local-engine/libch.so:/usr/lib/jvm/java-1.8.0-openjdk-amd64/jre/lib/amd64/libjsig.so` + +## Debug Native Code with CLion + +1. Toolchains Settings + + ![gluten-debug-clion-toolchains](../image/ClickHouse/gluten-debug-clion-toolchains.png) + > Some Clion versions have not supported lldb-18, you can manually set refer to [CLion Custom LLDB](https://youtrack.jetbrains.com/issue/CPP-3589/Support-using-custom-LLDB). Or you can use gdb as default. + +2. CMake Debug Configuration + + ![gluten-debug-clion-debug](../image/ClickHouse/gluten-debug-clion-debug.png) + + CMake Options: + `-DENABLE_PROTOBUF=ON -DENABLE_TESTS=OFF -DENABLE_BENCHMARKS=ON -DENABLE_JEMALLOC=ON -DENABLE_MULTITARGET_CODE=ON -DENABLE_EXTERN_LOCAL_ENGINE=ON -DENABLE_ODBC=OFF -DENABLE_CAPNP=OFF -DENABLE_ROCKSDB=OFF -DENABLE_GRPC=OFF -DENABLE_RUST=OFF -DENABLE_H3=OFF -DENABLE_AMQPCPP=OFF -DENABLE_CASSANDRA=OFF -DENABLE_KAFKA=OFF -DENABLE_NATS=OFF -DENABLE_LIBPQXX=OFF -DENABLE_NURAFT=OFF -DENABLE_DATASKETCHES=OFF -DENABLE_SQLITE=OFF -DENABLE_S2_GEOMETRY=OFF -DENABLE_ANNOY=OFF -DENABLE_ULID=OFF -DENABLE_MYSQL=OFF -DENABLE_BCRYPT=OFF -DENABLE_LDAP=OFF -DENABLE_MSGPACK=OFF -DUSE_REPLXX=OFF -DENABLE_CLICKHOUSE_ALL=OFF -DCOMPILER_FLAGS="-fvisibility=hidden -fvisibility-inlines-hidden" -DENABLE_BUILD_PATH_MAPPING=OFF -DDEBUG_0_LEVEL="0"` + +3. Reload CMake Project + + ![gluten-debug-cmake-reload](../image/ClickHouse/gluten-debug-cmake-reload.png) + + After reload cmake projects, you can find target `libch` in run configurations. + +4. Build `libchd.so` with Debug Mode + + ![gluten-debug-build-libch](../image/ClickHouse/gluten-debug-build-libch.png) + + `libchd.so` will be generated in `cmake-build-debug/utils/extern-local-engine/libchd.so`. + +5. Create File `.gdbinit` and `.lldbinit` to Avoid Unused Signal + + vi ~/.gdbinit + ``` + handle SIGSEGV nostop noprint + + set print pretty on + set print object on + python + import sys + sys.path.insert(0, '/path/to/libcxx-pretty-printers/src') + from libcxx.v1.printers import register_libcxx_printers + register_libcxx_printers(None) + end + ``` + > Download libcxx-pretty-printers in https://github.com/koutheir/libcxx-pretty-printers + + vi ~/.lldbinit + ``` + process handle -n true -p true -s false SIGBUS SIGSEGV + ``` + +6. Debug Application + + ![gluten-debug-cmake-debug.png](../image/ClickHouse/gluten-debug-cmake-debug.png) + + Executable: `/path/to/java` + + Program Arguments: Copy from IntelliJ IDEA debug command line and remove `-javaagent` + ![gluten-debug-program-args.png](../image/ClickHouse/gluten-debug-program-args.png) + + diff --git a/docs/developers/improvement-proposals.md b/docs/developers/improvement-proposals.md new file mode 100644 index 000000000000..1b7c47bbf222 --- /dev/null +++ b/docs/developers/improvement-proposals.md @@ -0,0 +1,100 @@ +--- +layout: page +title: Gluten Project Improvement Proposals (GPIP) +nav_order: 10 +parent: Developer Overview +--- + +# Gluten Project Improvement Proposals (GPIP) + +The Gluten Project Improvement Proposals doc references [the Spark SPIP documentation](https://spark.apache.org/improvement-proposals.html). + +The purpose of a GPIP is to inform and involve the user community in major improvements to the Gluten codebase throughout the development process to increase the likelihood that user needs are met. + +GPIPs should be used for significant user-facing or cutting-edge changes, not small incremental improvements. + +If your proposal meets the definition of GPIP, we recommend you to create a GPIP, which will facilitate the advancement and discussion of the proposal, but it is not mandatory, and we welcome any contribution and community participation. + +## What is a GPIP? + +A GPIP is similar to a product requirement document commonly used in product management. + +A GPIP: + +- Is a ticket labeled “GPIP” proposing a major improvement or change to Gluten +- Follows the template defined below +- Includes discussions on the ticket and dev@ list about the proposal + +## Who? + +Any **community member** can help by discussing whether a GPIP is likely to meet their needs and propose GPIPs. + +**Contributors** can help by discussing whether a GPIP is likely to be technically feasible. + +**Committers** can help by discussing whether a GPIP aligns with long-term project goals, and by shepherding GPIPs. + +**GPIP Author** is any community member who authors a GPIP and is committed to pushing the change through the entire process. GPIP authorship can be transferred. + +**GPIP Shepherd** is a PMC member who is committed to shepherding the proposed change throughout the entire process. Although the shepherd can delegate or work with other committers in the development process, the shepherd is ultimately responsible for the success or failure of the GPIP. Responsibilities of the shepherd include, but are not limited to: + +- Be the advocate for the proposed change +- Help push forward on design and achieve consensus among key stakeholders +- Review code changes, making sure the change follows project standards +- Get feedback from users and iterate on the design & implementation +- Uphold the quality of the changes, including verifying whether the changes satisfy the goal of the GPIP and are absent of critical bugs before releasing them + +## GPIP Process +### Proposing a GPIP + +Anyone may propose a GPIP, using the document template below. Please only submit a GPIP if you are willing to help, at least with discussion. + +After a GPIP is created, the author should email dev@gluten.apache.org to notify the community of the GPIP, and discussions should ensue on the ticket. + +If a GPIP is too small or incremental and should have been done through the normal ticket process, a committer should remove the GPIP label. + +### GPIP Document Template + +A GPIP document is a short document with a few questions, inspired by the Heilmeier Catechism: + +- Q1. What are you trying to do? Articulate your objectives using absolutely no jargon. + +- Q2. What problem is this proposal NOT designed to solve? + +- Q3. How is it done today, and what are the limits of current practice? + +- Q4. What is new in your approach, and why do you think it will be successful? + +- Q5. Who cares? If you are successful, what difference will it make? + +- Q6. What are the risks? + +- Q7. How long will it take? + +- Q8. What are the mid-term and final “exams” to check for success? + +- Appendix A. Proposed API Changes. Optional section defining APIs changes, if any. Backward and forward compatibility must be taken into account. + +- Appendix B. Optional Design Sketch: How are the goals going to be accomplished? Give sufficient technical detail to allow a contributor to judge whether it's likely to be feasible. Note that this is not a full design document. + +- Appendix C. Optional Rejected Designs: What alternatives were considered? Why were they rejected? If no alternatives have been considered, the problem needs more thought. + +### Discussing a GPIP + +All discussions of a GPIP should take place in a public forum, preferably the discussion attached to the ticket. Any discussion that happen offline should be made available online for the public via meeting notes summarizing the discussions. + +During this discussion, one or more shepherds should be identified among PMC members. + +Once the discussion settles, the shepherd(s) should call for a vote on the GPIP moving forward on the dev@ list. The vote should be open for at least 72 hours and follows the typical Apache vote process and passes upon consensus (at least 3 +1 votes from PMC members and no -1 votes from PMC members). dev@ should be notified of the vote result. + +If there does not exist at least one PMC member that is committed to shepherding the change within a month, the GPIP is rejected. + +If a committer does not think a GPIP aligns with long-term project goals, or is not practical at the point of proposal, the committer should -1 the GPIP explicitly and give technical justifications. + + +### Implementing a GPIP + +Implementation should take place via the contribution guidelines. Changes that require GPIPs typically also require design documents to be written and reviewed. + +### GPIP Community collaboration + +The gluten community has always been open to contributions and participation of all kinds, and feel free to contact the community if you have any questions about the GPIP process. diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index 2f6a912f4362..287747418294 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -118,7 +118,7 @@ cd $gluten_root/cpp-ch/local-engine/docker `build_directory` is a directory used as to store the intermediate files from compiling. It will use `current_dir>`/build as the default value if you don't provide it. -`output_directory` is used as to store the finaly output `libch.so`. The default value is `current_dir`/output if you don't provide it. +`output_directory` is used as to store the finally output `libch.so`. The default value is `current_dir`/output if you don't provide it. 3. (Option 1) Use CLion diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index 28754848658b..188a2c833509 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -120,7 +120,8 @@ shared libraries into another jar named `gluten-thirdparty-lib-$LINUX_OS-$VERSIO ## HDFS support -Hadoop hdfs support is ready via the [libhdfs3](https://github.com/apache/hawq/tree/master/depends/libhdfs3) library. The libhdfs3 provides native API for Hadoop I/O without the drawbacks of JNI. It also provides advanced authentication like Kerberos based. Please note this library has several dependencies which may require extra installations on Driver and Worker node. +Gluten supports dynamically loading both libhdfs.so and libhdfs3.so at runtime by using dlopen, allowing the JVM to load the appropriate shared library file as needed. This means you do not need to set the library path during the compilation phase. +To enable this functionality, you must set the JAVA_HOME and HADOOP_HOME environment variables. Gluten will then locate and load the ${HADOOP_HOME}/lib/native/libhdfs.so file at runtime. If you prefer to use libhdfs3.so instead, simply replace the ${HADOOP_HOME}/lib/native/libhdfs.so file with libhdfs3.so. ### Build with HDFS support @@ -131,7 +132,7 @@ cd /path/to/gluten ./dev/buildbundle-veloxbe.sh --enable_hdfs=ON ``` -### Configuration about HDFS support +### Configuration about HDFS support in Libhdfs3 HDFS uris (hdfs://host:port) will be extracted from a valid hdfs file path to initialize hdfs client, you do not need to specify it explicitly. @@ -172,7 +173,7 @@ You also need to add configuration to the "hdfs-site.xml" as below: ``` -### Kerberos support +### Kerberos support in libhdfs3 Here are two steps to enable kerberos. @@ -216,7 +217,7 @@ cd /path/to/gluten ./dev/buildbundle-veloxbe.sh --enable_s3=ON ``` -Currently there are several ways to asscess S3 in Spark. Please refer [Velox S3](VeloxS3.md) part for more detailed configurations +Currently there are several ways to access S3 in Spark. Please refer [Velox S3](VeloxS3.md) part for more detailed configurations ## Celeborn support @@ -271,7 +272,7 @@ spark.dynamicAllocation.enabled false ## Uniffle support -Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.0`. +Uniffle with velox backend supports [Uniffle](https://github.com/apache/incubator-uniffle) as remote shuffle service. Currently, the supported Uniffle versions are `0.9.1`. First refer to this URL(https://uniffle.apache.org/docs/intro) to get start with uniffle. diff --git a/docs/get-started/VeloxGCS.md b/docs/get-started/VeloxGCS.md index 453bee117c1e..09e0a927cab4 100644 --- a/docs/get-started/VeloxGCS.md +++ b/docs/get-started/VeloxGCS.md @@ -24,7 +24,7 @@ After these steps, no specific configuration is required for Gluten, since the a For workloads that need to be fully automated, manually authorizing can be problematic. For such cases it is better to use a json file with the credentials. This is described in the [instructions to configure a service account]https://cloud.google.com/sdk/docs/authorizing#service-account. -Such json file with the credetials can be passed to Gluten: +Such json file with the credentials can be passed to Gluten: ```sh spark.hadoop.fs.gs.auth.type SERVICE_ACCOUNT_JSON_KEYFILE @@ -49,5 +49,5 @@ spark.hadoop.fs.gs.http.max.retry // number of times to keep retrying unless a n For cases when a transient server error is detected, GCS can be configured to keep retrying until the retry loop exceeds a prescribed duration. ```sh -spark.hadoop.fs.gs.http.max.retry-time // a string representing the time keep retring (10s, 1m, etc). +spark.hadoop.fs.gs.http.max.retry-time // a string representing the time keep retrying (10s, 1m, etc). ``` \ No newline at end of file diff --git a/docs/get-started/build-guide.md b/docs/get-started/build-guide.md index 0ea901d68151..bd4ec9012b90 100644 --- a/docs/get-started/build-guide.md +++ b/docs/get-started/build-guide.md @@ -14,8 +14,7 @@ Please set them via `--`, e.g. `--build_type=Release`. | build_tests | Build gluten cpp tests. | OFF | | build_examples | Build udf example. | OFF | | build_benchmarks | Build gluten cpp benchmarks. | OFF | -| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | -| build_protobuf | Build protobuf lib. | OFF | +| enable_jemalloc_stats | Print jemalloc stats for debugging. | OFF | | enable_qat | Enable QAT for shuffle data de/compression. | OFF | | enable_iaa | Enable IAA for shuffle data de/compression. | OFF | | enable_hbm | Enable HBM allocator. | OFF | diff --git a/docs/image/ClickHouse/gluten-debug-build-libch.png b/docs/image/ClickHouse/gluten-debug-build-libch.png new file mode 100644 index 000000000000..4cf914dc8b2e Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-build-libch.png differ diff --git a/docs/image/ClickHouse/gluten-debug-clion-debug.png b/docs/image/ClickHouse/gluten-debug-clion-debug.png new file mode 100644 index 000000000000..e217eff0976b Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-clion-debug.png differ diff --git a/docs/image/ClickHouse/gluten-debug-clion-toolchains.png b/docs/image/ClickHouse/gluten-debug-clion-toolchains.png new file mode 100644 index 000000000000..a90e474463cb Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-clion-toolchains.png differ diff --git a/docs/image/ClickHouse/gluten-debug-cmake-debug.png b/docs/image/ClickHouse/gluten-debug-cmake-debug.png new file mode 100644 index 000000000000..3c050328c415 Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-cmake-debug.png differ diff --git a/docs/image/ClickHouse/gluten-debug-cmake-reload.png b/docs/image/ClickHouse/gluten-debug-cmake-reload.png new file mode 100644 index 000000000000..40d2c7775874 Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-cmake-reload.png differ diff --git a/docs/image/ClickHouse/gluten-debug-idea-config.png b/docs/image/ClickHouse/gluten-debug-idea-config.png new file mode 100644 index 000000000000..112b0d3fbdae Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-idea-config.png differ diff --git a/docs/image/ClickHouse/gluten-debug-profile-settings.png b/docs/image/ClickHouse/gluten-debug-profile-settings.png new file mode 100644 index 000000000000..b2b7a8451de7 Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-profile-settings.png differ diff --git a/docs/image/ClickHouse/gluten-debug-program-args.png b/docs/image/ClickHouse/gluten-debug-program-args.png new file mode 100644 index 000000000000..bc6475511f3d Binary files /dev/null and b/docs/image/ClickHouse/gluten-debug-program-args.png differ diff --git a/docs/release.md b/docs/release.md index a3f20bde857e..99661c07c9a3 100644 --- a/docs/release.md +++ b/docs/release.md @@ -6,10 +6,18 @@ nav_order: 11 [Gluten](https://github.com/apache/incubator-gluten) is a plugin for Apache Spark to double SparkSQL's performance. -## Latest release for velox backend -* [Gluten-1.1.1](https://github.com/apache/incubator-gluten/releases/tag/v1.1.1) (Mar. 2 2024) +## Latest release for Velox backend +* [Gluten-1.2.0](https://github.com/apache/incubator-gluten/releases/tag/v1.2.0) (Sep. 3 2024) + +## Planned release for Velox backend +* [Gluten-1.2.1] (Nov. 30 2024) +* [Gluten-1.3.0] (Dec. 30 2024) +* [Gluten-1.3.1] (Jan. 30 2025) +* [Gluten-1.4.0] (Mar. 30 2025) +* To be updated ## Archived releases +* [Gluten-1.1.1](https://github.com/apache/incubator-gluten/releases/tag/v1.1.1) (Mar. 2 2024) * [Gluten-1.1.0](https://github.com/apache/incubator-gluten/releases/tag/v1.1.0) (Nov. 30 2023) * [Gluten-1.0.0](https://github.com/apache/incubator-gluten/releases/tag/v1.0.0) (Jul. 13 2023) * [Gluten-0.5.0](https://github.com/apache/incubator-gluten/releases/tag/0.5.0) (Apr. 7 2023) diff --git a/docs/velox-backend-limitations.md b/docs/velox-backend-limitations.md index e05b589524d5..161b1bfc713a 100644 --- a/docs/velox-backend-limitations.md +++ b/docs/velox-backend-limitations.md @@ -147,7 +147,7 @@ Gluten's. ### Spill -`OutOfMemoryExcetpion` may still be triggered within current implementation of spill-to-disk feature, when shuffle partitions is set to a large number. When this case happens, please try to reduce the partition number to get rid of the OOM. +`OutOfMemoryException` may still be triggered within current implementation of spill-to-disk feature, when shuffle partitions is set to a large number. When this case happens, please try to reduce the partition number to get rid of the OOM. ### Unsupported Data type support in ParquetScan diff --git a/ep/build-clickhouse/src/package.sh b/ep/build-clickhouse/src/package.sh index 0d544a983fee..2583727b212e 100755 --- a/ep/build-clickhouse/src/package.sh +++ b/ep/build-clickhouse/src/package.sh @@ -90,7 +90,7 @@ function build_gluten_by_spark_version() { sv=$(echo "$spark_profile" | tr -d '.') echo "build gluten with spark ${spark_profile}, scala ${scala_version}" - mvn clean install -Pbackends-clickhouse -Pspark-"${spark_profile}" -Pscala-"${scala_version}" -Pceleborn -DskipTests -Dcheckstyle.skip + mvn clean install -Pbackends-clickhouse -Pspark-"${spark_profile}" -Pscala-"${scala_version}" -Pceleborn -Piceberg -DskipTests -Dcheckstyle.skip cp "${GLUTEN_SOURCE}"/backends-clickhouse/target/gluten-*-spark-"${spark_profile}"-jar-with-dependencies.jar "${PACKAGE_DIR_PATH}"/jars/spark"${sv}"/gluten.jar cp "${GLUTEN_SOURCE}"/gluten-celeborn/clickhouse/target/gluten-celeborn-clickhouse-"${PROJECT_VERSION}"-jar-with-dependencies.jar "${PACKAGE_DIR_PATH}"/jars/spark"${sv}" delta_version=$(mvn -q -Dexec.executable="echo" -Dexec.args='${delta.version}' -Pspark-"${spark_profile}" --non-recursive exec:exec) diff --git a/ep/build-velox/src/build_velox.sh b/ep/build-velox/src/build_velox.sh index b48f28a374b2..18599c0f8fa7 100755 --- a/ep/build-velox/src/build_velox.sh +++ b/ep/build-velox/src/build_velox.sh @@ -97,7 +97,7 @@ function compile { set -exu CXX_FLAGS='-Wno-missing-field-initializers' - COMPILE_OPTION="-DCMAKE_CXX_FLAGS=\"$CXX_FLAGS\" -DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=OFF -DVELOX_MONO_LIBRARY=ON" + COMPILE_OPTION="-DCMAKE_CXX_FLAGS=\"$CXX_FLAGS\" -DVELOX_ENABLE_PARQUET=ON -DVELOX_BUILD_TESTING=OFF -DVELOX_MONO_LIBRARY=ON -DVELOX_BUILD_RUNNER=OFF" if [ $BUILD_TEST_UTILS == "ON" ]; then COMPILE_OPTION="$COMPILE_OPTION -DVELOX_BUILD_TEST_UTILS=ON" fi diff --git a/ep/build-velox/src/get_velox.sh b/ep/build-velox/src/get_velox.sh index 6d53466b07e7..e880db163505 100755 --- a/ep/build-velox/src/get_velox.sh +++ b/ep/build-velox/src/get_velox.sh @@ -17,7 +17,7 @@ set -exu VELOX_REPO=https://github.com/oap-project/velox.git -VELOX_BRANCH=2024_11_02 +VELOX_BRANCH=2024_12_07 VELOX_HOME="" OS=`uname -s` @@ -69,17 +69,6 @@ function process_setup_ubuntu { # Do not install libunwind which can cause interruption when catching native exception. ensure_pattern_matched '\${SUDO} apt install -y libunwind-dev' scripts/setup-ubuntu.sh sed -i 's/${SUDO} apt install -y libunwind-dev//' scripts/setup-ubuntu.sh - # Overwrite gcc installed by build-essential. - ensure_pattern_matched '\${SUDO} pip3 install cmake==3.28.3' scripts/setup-ubuntu.sh - sed -i '/^ ${SUDO} pip3 install cmake==3.28.3/a\ - \VERSION=`cat /etc/os-release | grep VERSION_ID`\ - if [[ $VERSION =~ "20.04" ]]; then\ - sudo apt install -y software-properties-common\ - sudo add-apt-repository ppa:ubuntu-toolchain-r/test\ - sudo apt update && sudo apt install -y gcc-11 g++-11\ - sudo ln -sf /usr/bin/gcc-11 /usr/bin/gcc\ - sudo ln -sf /usr/bin/g++-11 /usr/bin/g++\ - fi' scripts/setup-ubuntu.sh ensure_pattern_matched 'ccache' scripts/setup-ubuntu.sh sed -i '/ccache/a\ *thrift* \\' scripts/setup-ubuntu.sh sed -i '/ccache/a\ libiberty-dev \\' scripts/setup-ubuntu.sh @@ -200,7 +189,6 @@ function setup_linux { local LINUX_VERSION_ID=$(. /etc/os-release && echo ${VERSION_ID}) # apply patches - sed -i 's/-mavx2 -mfma -mavx -mf16c -mlzcnt -std=c++17/-march=native -std=c++17 -mno-avx512f/g' scripts/setup-helper-functions.sh sed -i 's/SUDO="${SUDO:-""}"/SUDO="${SUDO:-"sudo --preserve-env"}"/g' scripts/setup-helper-functions.sh if [[ "$LINUX_DISTRIBUTION" == "ubuntu" || "$LINUX_DISTRIBUTION" == "debian" || "$LINUX_DISTRIBUTION" == "pop" ]]; then process_setup_ubuntu diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch index 3d45ff4b48a3..6938deddef99 100644 --- a/ep/build-velox/src/modify_velox.patch +++ b/ep/build-velox/src/modify_velox.patch @@ -1,66 +1,5 @@ -diff --git a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -index ddf1fac71..ef48ae9d9 100644 ---- a/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -+++ b/CMake/resolve_dependency_modules/arrow/CMakeLists.txt -@@ -14,17 +14,13 @@ - project(Arrow) - - if(VELOX_ENABLE_ARROW) -- find_package(Thrift) -- if(Thrift_FOUND) -- set(THRIFT_SOURCE "SYSTEM") -- else() -- set(THRIFT_SOURCE "BUNDLED") -- endif() -- - set(ARROW_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/arrow_ep") - set(ARROW_CMAKE_ARGS -- -DARROW_PARQUET=OFF -- -DARROW_DEPENDENCY_SOURCE=AUTO -+ -DARROW_PARQUET=ON -+ -DARROW_FILESYSTEM=ON -+ -DARROW_PROTOBUF_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_USE_SHARED=OFF -+ -DARROW_DEPENDENCY_SOURCE=BUNDLED - -DARROW_WITH_THRIFT=ON - -DARROW_WITH_LZ4=ON - -DARROW_WITH_SNAPPY=ON -@@ -38,18 +34,16 @@ if(VELOX_ENABLE_ARROW) - -DCMAKE_INSTALL_PREFIX=${ARROW_PREFIX}/install - -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} - -DARROW_BUILD_STATIC=ON -- -DThrift_SOURCE=${THRIFT_SOURCE} -+ -DARROW_BUILD_SHARED=OFF - -DCMAKE_PREFIX_PATH=${CMAKE_PREFIX_PATH}) - set(ARROW_LIBDIR ${ARROW_PREFIX}/install/${CMAKE_INSTALL_LIBDIR}) - - add_library(thrift STATIC IMPORTED GLOBAL) -- if(NOT Thrift_FOUND) -- set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -- set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) -+ set(THRIFT_ROOT ${ARROW_PREFIX}/src/arrow_ep-build/thrift_ep-install) -+ set(THRIFT_LIB ${THRIFT_ROOT}/lib/libthrift.a) - -- file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -- set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) -- endif() -+ file(MAKE_DIRECTORY ${THRIFT_ROOT}/include) -+ set(THRIFT_INCLUDE_DIR ${THRIFT_ROOT}/include) - - set_property(TARGET thrift PROPERTY INTERFACE_INCLUDE_DIRECTORIES - ${THRIFT_INCLUDE_DIR}) -@@ -68,6 +62,9 @@ if(VELOX_ENABLE_ARROW) - arrow_ep - PREFIX ${ARROW_PREFIX} - URL ${VELOX_ARROW_SOURCE_URL} -+ PATCH_COMMAND patch -p1 < ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow.patch -+ COMMAND patch -p1 < -+ ${CMAKE_CURRENT_SOURCE_DIR}/modify_arrow_dataset_scan_option.patch - URL_HASH ${VELOX_ARROW_BUILD_SHA256_CHECKSUM} - SOURCE_SUBDIR cpp - CMAKE_ARGS ${ARROW_CMAKE_ARGS} diff --git a/CMake/resolve_dependency_modules/simdjson.cmake b/CMake/resolve_dependency_modules/simdjson.cmake -index 69e7f2044..777eb5ec1 100644 +index ddd232050..b6ed2432b 100644 --- a/CMake/resolve_dependency_modules/simdjson.cmake +++ b/CMake/resolve_dependency_modules/simdjson.cmake @@ -29,4 +29,6 @@ FetchContent_Declare( @@ -71,40 +10,15 @@ index 69e7f2044..777eb5ec1 100644 + FetchContent_MakeAvailable(simdjson) diff --git a/CMakeLists.txt b/CMakeLists.txt -index 7f7cbc92f..52adb1250 100644 +index 5929473ae..e4e28c4f1 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt -@@ -386,7 +391,7 @@ resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) - # for reference. find_package(range-v3) +@@ -422,7 +422,7 @@ velox_resolve_dependency(Boost 1.77.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) + + velox_set_source(gflags) + +-velox_resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) ++velox_resolve_dependency(gflags) - set_source(gflags) --resolve_dependency(gflags COMPONENTS ${VELOX_GFLAGS_TYPE}) -+resolve_dependency(gflags) if(NOT TARGET gflags::gflags) # This is a bit convoluted, but we want to be able to use gflags::gflags as a - # target even when velox is built as a subproject which uses -diff --git a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -index a8b0df52a..254c56b70 100644 ---- a/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -+++ b/velox/connectors/hive/storage_adapters/abfs/RegisterAbfsFileSystem.cpp -@@ -38,7 +38,6 @@ std::shared_ptr abfsFileSystemGenerator( - - void registerAbfsFileSystem() { - #ifdef VELOX_ENABLE_ABFS -- LOG(INFO) << "Register ABFS"; - registerFileSystem(isAbfsFile, std::function(abfsFileSystemGenerator)); - #endif - } -diff --git a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -index 10ee508ba..027a58ecc 100644 ---- a/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -+++ b/velox/connectors/hive/storage_adapters/hdfs/tests/HdfsMiniCluster.cpp -@@ -72,7 +72,7 @@ HdfsMiniCluster::HdfsMiniCluster() { - "Failed to find minicluster executable {}'", miniClusterExecutableName); - } - boost::filesystem::path hadoopHomeDirectory = exePath_; -- hadoopHomeDirectory.remove_leaf().remove_leaf(); -+ hadoopHomeDirectory.remove_filename().remove_filename(); - setupEnvironment(hadoopHomeDirectory.string()); - } - diff --git a/ep/build-velox/src/setup-centos7.sh b/ep/build-velox/src/setup-centos7.sh index 87150634134b..45880161a4a5 100755 --- a/ep/build-velox/src/setup-centos7.sh +++ b/ep/build-velox/src/setup-centos7.sh @@ -242,7 +242,7 @@ dnf_install epel-release dnf-plugins-core # For ccache, ninja dnf_install ccache wget which libevent-devel \ yasm \ openssl-devel libzstd-devel lz4-devel double-conversion-devel \ - curl-devel libxml2-devel libgsasl-devel libuuid-devel patch + curl-devel libxml2-devel libgsasl-devel libuuid-devel patch libicu-devel $SUDO dnf remove -y gflags diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java index 94312a2cf53e..d69e84c3a1fc 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatchJniWrapper.java @@ -30,24 +30,26 @@ public static ColumnarBatchJniWrapper create(Runtime runtime) { return new ColumnarBatchJniWrapper(runtime); } - public native long createWithArrowArray(long cSchema, long cArray); + // Static methods. + public static native String getType(long batch); - public native long getForEmptySchema(int numRows); + public static native long numColumns(long batch); + + public static native long numRows(long batch); - public native String getType(long batch); + public static native long numBytes(long batch); - public native long numColumns(long batch); + public static native void exportToArrow(long batch, long cSchema, long cArray); - public native long numRows(long batch); + public static native void close(long batch); - public native long numBytes(long batch); + // Member methods in which native code relies on the backend's runtime API implementation. + public native long createWithArrowArray(long cSchema, long cArray); - public native void exportToArrow(long batch, long cSchema, long cArray); + public native long getForEmptySchema(int numRows); public native long select(long batch, int[] columnIndices); - public native void close(long batch); - @Override public long rtHandle() { return runtime.getHandle(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java index 04236884a1a2..3914fb155ec4 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/ColumnarBatches.java @@ -45,6 +45,7 @@ import scala.collection.JavaConverters; public final class ColumnarBatches { + private static final String INTERNAL_BACKEND_KIND = "internal"; private ColumnarBatches() {} @@ -108,8 +109,8 @@ static boolean isZeroColumnBatch(ColumnarBatch batch) { * This method will always return a velox based ColumnarBatch. This method will close the input * column batch. */ - public static ColumnarBatch select(ColumnarBatch batch, int[] columnIndices) { - final Runtime runtime = Runtimes.contextInstance("ColumnarBatches#select"); + public static ColumnarBatch select(String backendName, ColumnarBatch batch, int[] columnIndices) { + final Runtime runtime = Runtimes.contextInstance(backendName, "ColumnarBatches#select"); switch (identifyBatchType(batch)) { case LIGHT: final IndicatorVector iv = getIndicatorVector(batch); @@ -188,8 +189,8 @@ public static ColumnarBatch load(BufferAllocator allocator, ColumnarBatch input) ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema arrowSchema = ArrowSchema.allocateNew(allocator); CDataDictionaryProvider provider = new CDataDictionaryProvider()) { - ColumnarBatchJniWrapper.create(Runtimes.contextInstance("ColumnarBatches#load")) - .exportToArrow(iv.handle(), cSchema.memoryAddress(), cArray.memoryAddress()); + ColumnarBatchJniWrapper.exportToArrow( + iv.handle(), cSchema.memoryAddress(), cArray.memoryAddress()); Data.exportSchema( allocator, ArrowUtil.toArrowSchema(cSchema, allocator, provider), provider, arrowSchema); @@ -229,7 +230,10 @@ public static ColumnarBatch offload(BufferAllocator allocator, ColumnarBatch inp if (input.numCols() == 0) { throw new IllegalArgumentException("batch with zero columns cannot be offloaded"); } - final Runtime runtime = Runtimes.contextInstance("ColumnarBatches#offload"); + // Batch-offloading doesn't involve any backend-specific native code. Use the internal + // backend to store native batch references only. + final Runtime runtime = + Runtimes.contextInstance(INTERNAL_BACKEND_KIND, "ColumnarBatches#offload"); try (ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema cSchema = ArrowSchema.allocateNew(allocator)) { ArrowAbiUtil.exportFromSparkColumnarBatch(allocator, input, cSchema, cArray); @@ -383,11 +387,11 @@ private static IndicatorVector getIndicatorVector(ColumnarBatch input) { return (IndicatorVector) input.column(0); } - public static long getNativeHandle(ColumnarBatch batch) { + public static long getNativeHandle(String backendName, ColumnarBatch batch) { if (isZeroColumnBatch(batch)) { final ColumnarBatchJniWrapper jniWrapper = ColumnarBatchJniWrapper.create( - Runtimes.contextInstance("ColumnarBatches#getNativeHandle")); + Runtimes.contextInstance(backendName, "ColumnarBatches#getNativeHandle")); return jniWrapper.getForEmptySchema(batch.numRows()); } return getIndicatorVector(batch).handle(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java index 3f09a3619b3a..251925d035e6 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVector.java @@ -54,7 +54,7 @@ void release() { } if (refCnt.decrementAndGet() == 0) { pool.remove(handle); - jniWrapper.close(handle); + ColumnarBatchJniWrapper.close(handle); } } } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java index 700eb3cadeee..e0e266ca0490 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/columnarbatch/IndicatorVectorBase.java @@ -16,8 +16,6 @@ */ package org.apache.gluten.columnarbatch; -import org.apache.gluten.runtime.Runtimes; - import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.vectorized.ColumnVector; @@ -26,26 +24,23 @@ import org.apache.spark.unsafe.types.UTF8String; public abstract class IndicatorVectorBase extends ColumnVector { - protected final ColumnarBatchJniWrapper jniWrapper; protected final long handle; protected IndicatorVectorBase(long handle) { super(DataTypes.NullType); - this.jniWrapper = - ColumnarBatchJniWrapper.create(Runtimes.contextInstance("IndicatorVectorBase#init")); this.handle = handle; } public String getType() { - return jniWrapper.getType(handle); + return ColumnarBatchJniWrapper.getType(handle); } public long getNumColumns() { - return jniWrapper.numColumns(handle); + return ColumnarBatchJniWrapper.numColumns(handle); } public long getNumRows() { - return jniWrapper.numRows(handle); + return ColumnarBatchJniWrapper.numRows(handle); } abstract long refCnt(); diff --git a/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java b/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java index f85187fae695..fe1ebc763415 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/init/NativeBackendInitializer.java @@ -17,7 +17,6 @@ package org.apache.gluten.init; import org.apache.gluten.GlutenConfig; -import org.apache.gluten.backend.Backend; import org.apache.gluten.utils.ConfigUtil; import org.apache.spark.util.SparkShutdownManagerUtil; @@ -25,6 +24,7 @@ import org.slf4j.LoggerFactory; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; import scala.runtime.BoxedUnit; @@ -32,7 +32,18 @@ // Initialize native backend before calling any native methods from Java side. public final class NativeBackendInitializer { private static final Logger LOG = LoggerFactory.getLogger(NativeBackendInitializer.class); - private static final AtomicBoolean initialized = new AtomicBoolean(false); + private static final Map instances = new ConcurrentHashMap<>(); + + private final AtomicBoolean initialized = new AtomicBoolean(false); + private final String backendName; + + private NativeBackendInitializer(String backendName) { + this.backendName = backendName; + } + + public static NativeBackendInitializer forBackend(String backendName) { + return instances.computeIfAbsent(backendName, k -> new NativeBackendInitializer(backendName)); + } // Spark DriverPlugin/ExecutorPlugin will only invoke NativeBackendInitializer#initializeBackend // method once in its init method. @@ -41,7 +52,7 @@ public final class NativeBackendInitializer { // In local mode, NativeBackendInitializer#initializeBackend will be invoked twice in same // thread, driver first then executor, initialized flag ensure only invoke initializeBackend once, // so there are no race condition here. - public static void initializeBackend(scala.collection.Map conf) { + public void initialize(scala.collection.Map conf) { if (!initialized.compareAndSet(false, true)) { // Already called. return; @@ -54,10 +65,9 @@ public static void initializeBackend(scala.collection.Map conf) }); } - private static void initialize0(scala.collection.Map conf) { + private void initialize0(scala.collection.Map conf) { try { - Map nativeConfMap = - GlutenConfig.getNativeBackendConf(Backend.get().name(), conf); + Map nativeConfMap = GlutenConfig.getNativeBackendConf(backendName, conf); initialize(ConfigUtil.serialize(nativeConfMap)); } catch (Exception e) { LOG.error("Failed to call native backend's initialize method", e); @@ -65,9 +75,7 @@ private static void initialize0(scala.collection.Map conf) { } } - private static native void initialize(byte[] configPlan); - - private static native void shutdown(); + private native void initialize(byte[] configPlan); - private NativeBackendInitializer() {} + private native void shutdown(); } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java index f95324fad991..378927bec433 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ColumnarBatchInIterator.java @@ -23,9 +23,11 @@ import java.util.Iterator; public class ColumnarBatchInIterator { + private final String backendName; private final Iterator delegated; - public ColumnarBatchInIterator(Iterator delegated) { + public ColumnarBatchInIterator(String backendName, Iterator delegated) { + this.backendName = backendName; this.delegated = delegated; } @@ -38,6 +40,6 @@ public boolean hasNext() { public long next() { final ColumnarBatch next = delegated.next(); ColumnarBatches.checkOffloaded(next); - return ColumnarBatches.getNativeHandle(next); + return ColumnarBatches.getNativeHandle(backendName, next); } } diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java index 1c03c415ed10..bfa059b5aeb2 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/NativePlanEvaluator.java @@ -32,17 +32,19 @@ public class NativePlanEvaluator { private static final AtomicInteger id = new AtomicInteger(0); - private final Runtime runtime = - Runtimes.contextInstance(String.format("NativePlanEvaluator-%d", id.getAndIncrement())); + private final Runtime runtime; private final PlanEvaluatorJniWrapper jniWrapper; - private NativePlanEvaluator() { - jniWrapper = PlanEvaluatorJniWrapper.create(runtime); + private NativePlanEvaluator(Runtime runtime) { + this.runtime = runtime; + this.jniWrapper = PlanEvaluatorJniWrapper.create(runtime); } - public static NativePlanEvaluator create() { - return new NativePlanEvaluator(); + public static NativePlanEvaluator create(String backendName) { + return new NativePlanEvaluator( + Runtimes.contextInstance( + backendName, String.format("NativePlanEvaluator-%d", id.getAndIncrement()))); } public NativePlanValidationInfo doNativeValidateWithFailureReason(byte[] subPlan) { diff --git a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java index 09f97cbd06b9..8d031cf0792c 100644 --- a/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java +++ b/gluten-arrow/src/main/java/org/apache/gluten/vectorized/ShuffleReaderJniWrapper.java @@ -40,6 +40,7 @@ public native long make( String compressionType, String compressionCodecBackend, int batchSize, + long bufferSize, String shuffleWriterType); public native long readStream(long shuffleReaderHandle, JniByteInputStream jniIn); diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala b/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala index 5ae3863c57a8..c23d6eea794e 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/columnarbatch/ArrowBatches.scala @@ -33,7 +33,9 @@ object ArrowBatches { * implementations. */ object ArrowJavaBatch extends Convention.BatchType { - toBatch(VanillaBatch, Transition.empty) + override protected def registerTransitions(): Unit = { + toBatch(VanillaBatch, Transition.empty) + } } /** @@ -44,7 +46,9 @@ object ArrowBatches { * [[ColumnarBatches]]. */ object ArrowNativeBatch extends Convention.BatchType { - fromBatch(ArrowJavaBatch, OffloadArrowDataExec.apply) - toBatch(ArrowJavaBatch, LoadArrowDataExec.apply) + override protected def registerTransitions(): Unit = { + fromBatch(ArrowJavaBatch, OffloadArrowDataExec.apply) + toBatch(ArrowJavaBatch, LoadArrowDataExec.apply) + } } } diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala b/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala index 409c4297fac2..e65d3c4d9589 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/memory/NativeMemoryManager.scala @@ -17,7 +17,6 @@ package org.apache.gluten.memory import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.exception.GlutenException import org.apache.gluten.memory.listener.ReservationListeners import org.apache.gluten.memory.memtarget.{KnownNameAndStats, MemoryTarget, Spiller, Spillers} @@ -42,18 +41,19 @@ trait NativeMemoryManager { } object NativeMemoryManager { - private class Impl(name: String) extends NativeMemoryManager with TaskResource { + private class Impl(backendName: String, name: String) + extends NativeMemoryManager + with TaskResource { private val LOGGER = LoggerFactory.getLogger(classOf[NativeMemoryManager]) private val spillers = Spillers.appendable() private val mutableStats: mutable.Map[String, MemoryUsageStatsBuilder] = mutable.Map() private val rl = ReservationListeners.create(name, spillers, mutableStats.asJava) private val handle = NativeMemoryManagerJniWrapper.create( - Backend.get().name(), + backendName, rl, ConfigUtil.serialize( - GlutenConfig.getNativeSessionConf( - Backend.get().name(), - GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) + GlutenConfig + .getNativeSessionConf(backendName, GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) ) spillers.append(new Spiller() { override def spill(self: MemoryTarget, phase: Spiller.Phase, size: Long): Long = { @@ -109,11 +109,15 @@ object NativeMemoryManager { )) } } - override def priority(): Int = 0 + override def priority(): Int = { + // Memory managers should be released after all runtimes are released. + // So lower the priority to 0. + 0 + } override def resourceName(): String = "nmm" } - def apply(name: String): NativeMemoryManager = { - TaskResources.addAnonymousResource(new Impl(name)) + def apply(backendName: String, name: String): NativeMemoryManager = { + TaskResources.addAnonymousResource(new Impl(backendName, name)) } } diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala index efd4928cfdee..8741c12474e4 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtime.scala @@ -17,7 +17,6 @@ package org.apache.gluten.runtime import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.exception.GlutenException import org.apache.gluten.memory.NativeMemoryManager import org.apache.gluten.utils.ConfigUtil @@ -35,21 +34,20 @@ trait Runtime { } object Runtime { - private[runtime] def apply(name: String): Runtime with TaskResource = { - new RuntimeImpl(name) + private[runtime] def apply(backendName: String, name: String): Runtime with TaskResource = { + new RuntimeImpl(backendName, name) } - private class RuntimeImpl(name: String) extends Runtime with TaskResource { + private class RuntimeImpl(backendName: String, name: String) extends Runtime with TaskResource { private val LOGGER = LoggerFactory.getLogger(classOf[Runtime]) - private val nmm: NativeMemoryManager = NativeMemoryManager(name) + private val nmm: NativeMemoryManager = NativeMemoryManager(backendName, name) private val handle = RuntimeJniWrapper.createRuntime( - Backend.get().name(), + backendName, nmm.getHandle(), ConfigUtil.serialize( - GlutenConfig.getNativeSessionConf( - Backend.get().name(), - GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) + GlutenConfig + .getNativeSessionConf(backendName, GlutenConfigUtil.parseConfig(SQLConf.get.getAllConfs))) ) private val released: AtomicBoolean = new AtomicBoolean(false) diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala index bfb2465b12b9..1aca275d5ce9 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/runtime/Runtimes.scala @@ -21,15 +21,16 @@ import org.apache.spark.task.{TaskResource, TaskResources} object Runtimes { /** Get or create the runtime which bound with Spark TaskContext. */ - def contextInstance(name: String): Runtime = { + def contextInstance(backendName: String, name: String): Runtime = { if (!TaskResources.inSparkTask()) { throw new IllegalStateException("This method must be called in a Spark task.") } - TaskResources.addResourceIfNotRegistered(name, () => create(name)) + val resourceName = String.format("%s:%s", backendName, name) + TaskResources.addResourceIfNotRegistered(resourceName, () => create(backendName, name)) } - private def create(name: String): Runtime with TaskResource = { - Runtime(name) + private def create(backendName: String, name: String): Runtime with TaskResource = { + Runtime(backendName, name) } } diff --git a/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala b/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala index dea0a4814028..f9bb7478e7d0 100644 --- a/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala +++ b/gluten-arrow/src/main/scala/org/apache/gluten/utils/DebugUtil.scala @@ -21,24 +21,32 @@ import org.apache.gluten.GlutenConfig import org.apache.spark.TaskContext object DebugUtil { - // if specify taskId, then only do that task partition - // if not specify stageId, then do nothing + // if taskId is specified and matches, then do that task + // if stageId is not specified or doesn't match, then do nothing // if specify stageId but no partitionId, then do all partitions for that stage // if specify stageId and partitionId, then only do that partition for that stage def saveInputToFile(): Boolean = { - if (TaskContext.get().taskAttemptId() == GlutenConfig.getConf.taskId) { - return true - } - if (TaskContext.get().stageId() != GlutenConfig.getConf.taskStageId) { - return false - } - if (GlutenConfig.getConf.taskPartitionId == -1) { - return true - } - if (TaskContext.getPartitionId() == GlutenConfig.getConf.taskPartitionId) { - return true - } + def taskIdMatches = + GlutenConfig.getConf.benchmarkTaskId.nonEmpty && + GlutenConfig.getConf.benchmarkTaskId + .split(",") + .map(_.toLong) + .contains(TaskContext.get().taskAttemptId()) + + def partitionIdMatches = + TaskContext.get().stageId() == GlutenConfig.getConf.benchmarkStageId && + (GlutenConfig.getConf.benchmarkPartitionId.isEmpty || + GlutenConfig.getConf.benchmarkPartitionId + .split(",") + .map(_.toInt) + .contains(TaskContext.get().partitionId())) - false + val saveInput = taskIdMatches || partitionIdMatches + if (saveInput) { + if (GlutenConfig.getConf.benchmarkSaveDir.isEmpty) { + throw new IllegalArgumentException(GlutenConfig.BENCHMARK_SAVE_DIR.key + " is not set.") + } + } + saveInput } } diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala index 11c45264dbc4..2263cda7eafb 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriter.scala @@ -50,7 +50,8 @@ class CHCelebornColumnarShuffleWriter[K, V]( writeMetrics) { private val capitalizedCompressionCodec = - customizedCompressionCodec.toUpperCase(Locale.ROOT) + if (customizedCompressionCodec != null) customizedCompressionCodec.toUpperCase(Locale.ROOT) + else "NONE" private val jniWrapper = new CHShuffleSplitterJniWrapper diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala index 42dcd9bed567..e4652c541512 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornColumnarShuffleWriterFactory.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.shuffle -import org.apache.gluten.backend.Backend +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.spark.TaskContext import org.apache.spark.shuffle.celeborn.CelebornShuffleHandle @@ -26,7 +26,7 @@ import org.apache.celeborn.client.ShuffleClient import org.apache.celeborn.common.CelebornConf class CHCelebornColumnarShuffleWriterFactory extends CelebornShuffleWriterFactory { - override def backendName(): String = Backend.get().name() + override def backendName(): String = BackendsApiManager.getBackendName override def createShuffleWriterInstance[K, V]( shuffleId: Int, diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala index a1fb190e4593..a4a97d43de08 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarBatchSerializer.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig import org.apache.gluten.GlutenConfig.{GLUTEN_RSS_SORT_SHUFFLE_WRITER, GLUTEN_SORT_SHUFFLE_WRITER} +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.memory.arrow.alloc.ArrowBufferAllocators import org.apache.gluten.runtime.Runtimes import org.apache.gluten.utils.ArrowAbiUtil @@ -65,7 +66,8 @@ private class CelebornColumnarBatchSerializerInstance( extends SerializerInstance with Logging { - private val runtime = Runtimes.contextInstance("CelebornShuffleReader") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "CelebornShuffleReader") private val shuffleReaderHandle = { val allocator: BufferAllocator = ArrowBufferAllocators @@ -88,12 +90,14 @@ private class CelebornColumnarBatchSerializerInstance( .replace(GLUTEN_SORT_SHUFFLE_WRITER, GLUTEN_RSS_SORT_SHUFFLE_WRITER) val jniWrapper = ShuffleReaderJniWrapper.create(runtime) val batchSize = GlutenConfig.getConf.maxBatchSize + val bufferSize = GlutenConfig.getConf.columnarShuffleReaderBufferSize val handle = jniWrapper .make( cSchema.memoryAddress(), compressionCodec, compressionCodecBackend, batchSize, + bufferSize, shuffleWriterType ) // Close shuffle reader instance as lately as the end of task processing, diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala index 1a0bc475d327..165d68785d4b 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.columnarbatch.ColumnarBatches import org.apache.gluten.memory.memtarget.{MemoryTarget, Spiller, Spillers} import org.apache.gluten.runtime.Runtimes @@ -51,7 +52,8 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( writeMetrics) { private val isSort = !GlutenConfig.GLUTEN_HASH_SHUFFLE_WRITER.equals(shuffleWriterType) - private val runtime = Runtimes.contextInstance("CelebornShuffleWriter") + private val runtime = + Runtimes.contextInstance(BackendsApiManager.getBackendName, "CelebornShuffleWriter") private val jniWrapper = ShuffleWriterJniWrapper.create(runtime) @@ -75,7 +77,7 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( logInfo(s"Skip ColumnarBatch of ${cb.numRows} rows, ${cb.numCols} cols") } else { initShuffleWriter(cb) - val handle = ColumnarBatches.getNativeHandle(cb) + val handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, cb) val startTime = System.nanoTime() jniWrapper.write(nativeShuffleWriter, cb.numRows, handle, availableOffHeapPerTask()) dep.metrics("shuffleWallTime").add(System.nanoTime() - startTime) @@ -131,7 +133,7 @@ class VeloxCelebornColumnarShuffleWriter[K, V]( clientPushBufferMaxSize, clientPushSortMemoryThreshold, celebornPartitionPusher, - ColumnarBatches.getNativeHandle(columnarBatch), + ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName, columnarBatch), context.taskAttemptId(), GlutenShuffleUtils.getStartPartitionId(dep.nativePartitioning, context.partitionId), "celeborn", diff --git a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala index d5f5fd2e2150..03d16c41c72b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/GlutenPlugin.scala @@ -18,7 +18,7 @@ package org.apache.gluten import org.apache.gluten.GlutenBuildInfo._ import org.apache.gluten.GlutenConfig._ -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component import org.apache.gluten.events.GlutenBuildInfoEvent import org.apache.gluten.exception.GlutenException import org.apache.gluten.extension.GlutenSessionExtensions @@ -65,10 +65,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { postBuildInfoEvent(sc) - setPredefinedConfigs(sc, conf) + setPredefinedConfigs(conf) // Initialize Backend. - Backend.get().onDriverStart(sc, pluginContext) + Component.sorted().foreach(_.onDriverStart(sc, pluginContext)) Collections.emptyMap() } @@ -84,12 +84,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { } override def shutdown(): Unit = { - Backend.get().onDriverShutdown() + Component.sorted().reverse.foreach(_.onDriverShutdown()) } private def postBuildInfoEvent(sc: SparkContext): Unit = { - val buildInfo = Backend.get().buildInfo() - // export gluten version to property to spark System.setProperty("gluten.version", VERSION) @@ -105,10 +103,16 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { glutenBuildInfo.put("Gluten Revision Time", REVISION_TIME) glutenBuildInfo.put("Gluten Build Time", BUILD_DATE) glutenBuildInfo.put("Gluten Repo URL", REPO_URL) - glutenBuildInfo.put("Backend", buildInfo.name) - glutenBuildInfo.put("Backend Branch", buildInfo.branch) - glutenBuildInfo.put("Backend Revision", buildInfo.revision) - glutenBuildInfo.put("Backend Revision Time", buildInfo.revisionTime) + + Component.sorted().foreach { + comp => + val buildInfo = comp.buildInfo() + glutenBuildInfo.put("Component", buildInfo.name) + glutenBuildInfo.put("Component Branch", buildInfo.branch) + glutenBuildInfo.put("Component Revision", buildInfo.revision) + glutenBuildInfo.put("Component Revision Time", buildInfo.revisionTime) + } + val infoMap = glutenBuildInfo.toMap val loggingInfo = infoMap.toSeq .sortBy(_._1) @@ -123,7 +127,7 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { GlutenEventUtils.post(sc, event) } - private def setPredefinedConfigs(sc: SparkContext, conf: SparkConf): Unit = { + private def setPredefinedConfigs(conf: SparkConf): Unit = { // Spark SQL extensions val extensions = if (conf.contains(SPARK_SESSION_EXTENSIONS.key)) { s"${conf.get(SPARK_SESSION_EXTENSIONS.key)}," + @@ -134,7 +138,10 @@ private[gluten] class GlutenDriverPlugin extends DriverPlugin with Logging { conf.set(SPARK_SESSION_EXTENSIONS.key, extensions) // adaptive custom cost evaluator class - if (GlutenConfig.getConf.enableGluten && GlutenConfig.getConf.enableGlutenCostEvaluator) { + val enableGlutenCostEvaluator = conf.getBoolean( + GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED, + GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) + if (enableGlutenCostEvaluator) { val costEvaluator = "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator" conf.set(SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key, costEvaluator) } @@ -251,12 +258,12 @@ private[gluten] class GlutenExecutorPlugin extends ExecutorPlugin { /** Initialize the executor plugin. */ override def init(ctx: PluginContext, extraConf: util.Map[String, String]): Unit = { // Initialize Backend. - Backend.get().onExecutorStart(ctx) + Component.sorted().foreach(_.onExecutorStart(ctx)) } /** Clean up and terminate this plugin. For example: close the native engine. */ override def shutdown(): Unit = { - Backend.get().onExecutorShutdown() + Component.sorted().reverse.foreach(_.onExecutorShutdown()) super.shutdown() } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala index ef22d97e773b..02a2a44349c9 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Backend.scala @@ -16,63 +16,11 @@ */ package org.apache.gluten.backend -import org.apache.gluten.extension.columnar.transition.{Convention, ConventionFunc} -import org.apache.gluten.extension.injector.RuleInjector - -import org.apache.spark.SparkContext -import org.apache.spark.api.plugin.PluginContext - -import java.util.ServiceLoader - -import scala.collection.JavaConverters - -trait Backend { - import Backend._ - - /** Base information. */ - def name(): String - def buildInfo(): BuildInfo - - /** Spark listeners. */ - def onDriverStart(sc: SparkContext, pc: PluginContext): Unit = {} - def onDriverShutdown(): Unit = {} - def onExecutorStart(pc: PluginContext): Unit = {} - def onExecutorShutdown(): Unit = {} - - /** The columnar-batch type this backend is by default using. */ - def defaultBatchType: Convention.BatchType +trait Backend extends Component { /** - * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to - * determine the convention (its row-based processing / columnar-batch processing support) of a - * plan with a user-defined function that accepts a plan then returns convention type it outputs, - * and input conventions it requires. + * Backends don't have dependencies. They are all considered root components in the component DAG + * and will be loaded at the beginning. */ - def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty - - /** Query planner rules. */ - def injectRules(injector: RuleInjector): Unit -} - -object Backend { - private val backend: Backend = { - val discoveredBackends = - JavaConverters.iterableAsScalaIterable(ServiceLoader.load(classOf[Backend])).toList - discoveredBackends match { - case Nil => - throw new IllegalStateException("Backend implementation not discovered from JVM classpath") - case head :: Nil => - head - case backends => - val backendNames = backends.map(_.name()) - throw new IllegalStateException( - s"More than one Backend implementation discovered from JVM classpath: $backendNames") - } - } - - def get(): Backend = { - backend - } - - case class BuildInfo(name: String, branch: String, revision: String, revisionTime: String) + final override def dependencies(): Seq[Class[_ <: Component]] = Nil } diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala new file mode 100644 index 000000000000..8670bede87e5 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/Component.scala @@ -0,0 +1,251 @@ +/* + * 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.gluten.backend + +import org.apache.gluten.extension.columnar.transition.ConventionFunc +import org.apache.gluten.extension.injector.Injector + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.plugin.PluginContext + +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + +import scala.collection.mutable + +/** + * The base API to inject user-defined logic to Gluten. To register a component, its implementations + * should be placed to Gluten's classpath with a Java service file. Gluten will discover all the + * component implementations then register them at the booting time. + * + * Experimental: This is not expected to be used in production yet. Use [[Backend]] instead. + */ +@Experimental +trait Component { + import Component._ + + private val uid = nextUid.getAndIncrement() + private val isRegistered = new AtomicBoolean(false) + + def ensureRegistered(): Unit = { + if (!isRegistered.compareAndSet(false, true)) { + return + } + graph.add(this) + dependencies().foreach(req => graph.declareDependency(this, req)) + } + + /** Base information. */ + def name(): String + def buildInfo(): BuildInfo + def dependencies(): Seq[Class[_ <: Component]] + + /** Spark listeners. */ + def onDriverStart(sc: SparkContext, pc: PluginContext): Unit = {} + def onDriverShutdown(): Unit = {} + def onExecutorStart(pc: PluginContext): Unit = {} + def onExecutorShutdown(): Unit = {} + + /** + * Overrides [[org.apache.gluten.extension.columnar.transition.ConventionFunc]] Gluten is using to + * determine the convention (its row-based processing / columnar-batch processing support) of a + * plan with a user-defined function that accepts a plan then returns convention type it outputs, + * and input conventions it requires. + */ + def convFuncOverride(): ConventionFunc.Override = ConventionFunc.Override.Empty + + /** Query planner rules. */ + def injectRules(injector: Injector): Unit +} + +object Component { + private val nextUid = new AtomicInteger() + private val graph: Graph = new Graph() + + // format: off + /** + * Apply topology sort on all registered components in graph to get an ordered list of + * components. The root nodes will be on the head side of the list, while leaf nodes + * will be on the tail side of the list. + * + * Say if component-A depends on component-B while component-C requires nothing, then the + * output order will be one of the following: + * + * 1. [component-B, component-A, component-C] + * 2. [component-C, component-B, component-A] + * 3. [component-B, component-C, component-A] + * + * By all means component B will be placed before component A because of the declared + * dependency from component A to component B. + * + * @throws UnsupportedOperationException When cycles in dependency graph are found. + */ + // format: on + def sorted(): Seq[Component] = { + ensureAllComponentsRegistered() + graph.sorted() + } + + private[backend] def sortedUnsafe(): Seq[Component] = { + graph.sorted() + } + + private class Registry { + private val lookupByUid: mutable.Map[Int, Component] = mutable.Map() + private val lookupByClass: mutable.Map[Class[_ <: Component], Component] = mutable.Map() + + def register(comp: Component): Unit = synchronized { + val uid = comp.uid + val clazz = comp.getClass + require(!lookupByUid.contains(uid), s"Component UID $uid already registered: ${comp.name()}") + require( + !lookupByClass.contains(clazz), + s"Component class $clazz already registered: ${comp.name()}") + lookupByUid += uid -> comp + lookupByClass += clazz -> comp + } + + def isUidRegistered(uid: Int): Boolean = synchronized { + lookupByUid.contains(uid) + } + + def isClassRegistered(clazz: Class[_ <: Component]): Boolean = synchronized { + lookupByClass.contains(clazz) + } + + def findByClass(clazz: Class[_ <: Component]): Component = synchronized { + require(lookupByClass.contains(clazz)) + lookupByClass(clazz) + } + + def findByUid(uid: Int): Component = synchronized { + require(lookupByUid.contains(uid)) + lookupByUid(uid) + } + + def allUids(): Seq[Int] = synchronized { + return lookupByUid.keys.toSeq + } + } + + private class Graph { + import Graph._ + private val registry: Registry = new Registry() + private val dependencies: mutable.Buffer[(Int, Class[_ <: Component])] = mutable.Buffer() + + private var sortedComponents: Option[Seq[Component]] = None + + def add(comp: Component): Unit = synchronized { + require( + !registry.isUidRegistered(comp.uid), + s"Component UID ${comp.uid} already registered: ${comp.name()}") + require( + !registry.isClassRegistered(comp.getClass), + s"Component class ${comp.getClass} already registered: ${comp.name()}") + registry.register(comp) + sortedComponents = None + } + + def declareDependency(comp: Component, dependencyCompClass: Class[_ <: Component]): Unit = + synchronized { + require(registry.isUidRegistered(comp.uid)) + require(registry.isClassRegistered(comp.getClass)) + dependencies += comp.uid -> dependencyCompClass + sortedComponents = None + } + + private def newLookup(): mutable.Map[Int, Node] = { + val lookup: mutable.Map[Int, Node] = mutable.Map() + + registry.allUids().foreach { + uid => + require(!lookup.contains(uid)) + val n = new Node(uid) + lookup += uid -> n + } + + dependencies.foreach { + case (uid, dependencyCompClass) => + val dependencyUid = registry.findByClass(dependencyCompClass).uid + require(uid != dependencyUid) + require(lookup.contains(uid)) + require(lookup.contains(dependencyUid)) + val n = lookup(uid) + val r = lookup(dependencyUid) + require(!n.parents.contains(r.uid)) + require(!r.children.contains(n.uid)) + n.parents += r.uid -> r + r.children += n.uid -> n + } + + lookup + } + + def sorted(): Seq[Component] = synchronized { + if (sortedComponents.isDefined) { + return sortedComponents.get + } + + val lookup: mutable.Map[Int, Node] = newLookup() + + val out = mutable.Buffer[Component]() + val uidToNumParents = lookup.map { case (uid, node) => uid -> node.parents.size } + val removalQueue = mutable.Queue[Int]() + + // 1. Find out all nodes with zero parents then enqueue them. + uidToNumParents.filter(_._2 == 0).foreach(kv => removalQueue.enqueue(kv._1)) + + // 2. Loop to dequeue and remove nodes from the uid-to-num-parents map. + while (removalQueue.nonEmpty) { + val parentUid = removalQueue.dequeue() + val node = lookup(parentUid) + out += registry.findByUid(parentUid) + node.children.keys.foreach { + childUid => + uidToNumParents += childUid -> (uidToNumParents(childUid) - 1) + val updatedNumParents = uidToNumParents(childUid) + assert(updatedNumParents >= 0) + if (updatedNumParents == 0) { + removalQueue.enqueue(childUid) + } + } + } + + // 3. If there are still outstanding nodes (those are with more non-zero parents) in the + // uid-to-num-parents map, then it means at least one cycle is found. Report error if so. + if (uidToNumParents.exists(_._2 != 0)) { + val cycleNodes = uidToNumParents.filter(_._2 != 0).keys.map(registry.findByUid) + val cycleNodeNames = cycleNodes.map(_.name()).mkString(", ") + throw new UnsupportedOperationException( + s"Cycle detected in the component graph: $cycleNodeNames") + } + + // 4. Return the ordered nodes. + sortedComponents = Some(out.toSeq) + sortedComponents.get + } + } + + private object Graph { + class Node(val uid: Int) { + val parents: mutable.Map[Int, Node] = mutable.Map() + val children: mutable.Map[Int, Node] = mutable.Map() + } + } + + case class BuildInfo(name: String, branch: String, revision: String, revisionTime: String) +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala b/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala new file mode 100644 index 000000000000..a9981719a333 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/backend/package.scala @@ -0,0 +1,47 @@ +/* + * 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.gluten + +import org.apache.spark.internal.Logging + +import java.util.ServiceLoader +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +package object backend extends Logging { + private[backend] val allComponentsLoaded: AtomicBoolean = new AtomicBoolean(false) + + private[backend] def ensureAllComponentsRegistered(): Unit = { + if (!allComponentsLoaded.compareAndSet(false, true)) { + return + } + + // Load all components in classpath. + val discoveredBackends = ServiceLoader.load(classOf[Backend]).asScala + val discoveredComponents = ServiceLoader.load(classOf[Component]).asScala + val all = discoveredBackends ++ discoveredComponents + + // Register all components. + all.foreach(_.ensureRegistered()) + + // Output log so user could view the component loading order. + // Call #sortedUnsafe than on #sorted to avoid unnecessary recursion. + val components = Component.sortedUnsafe() + logInfo(s"Components registered within order: ${components.map(_.name()).mkString(", ")}") + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala index d0eacc1e4dac..ac1f8d683570 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/ColumnarToColumnarExec.scala @@ -17,8 +17,6 @@ package org.apache.gluten.execution import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.KnownBatchType -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions import org.apache.gluten.iterator.Iterators import org.apache.spark.rdd.RDD @@ -32,8 +30,7 @@ import java.util.concurrent.atomic.AtomicLong abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention.BatchType) extends ColumnarToColumnarTransition - with KnownBatchType - with KnownChildrenConventions { + with GlutenPlan { def child: SparkPlan protected def mapIterator(in: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] @@ -47,10 +44,15 @@ abstract class ColumnarToColumnarExec(from: Convention.BatchType, to: Convention "selfTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to convert batches") ) - override def supportsColumnar: Boolean = true override def batchType(): Convention.BatchType = to - override def requiredChildrenConventions(): Seq[ConventionReq] = List( - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(from))) + + override def rowType0(): Convention.RowType = { + Convention.RowType.None + } + + override def requiredChildConvention(): Seq[ConventionReq] = { + List(ConventionReq.ofBatch(ConventionReq.BatchType.Is(from))) + } override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala new file mode 100644 index 000000000000..2cd408f67c3f --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/GlutenPlan.scala @@ -0,0 +1,81 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} + +import org.apache.spark.sql.execution.SparkPlan + +/** + * Base interface for Query plan that defined by backends. + * + * The following Spark APIs are marked final so forbidden from overriding: + * - supportsColumnar + * - supportsRowBased (Spark version >= 3.3) + * + * Instead, subclasses are expected to implement the following APIs: + * - batchType + * - rowType0 + * - requiredChildConvention (optional) + * + * With implementations of the APIs provided, Gluten query planner will be able to find and insert + * proper transitions between different plan nodes. + * + * Implementing `requiredChildConvention` is optional while the default implementation is a sequence + * of convention reqs that are exactly the same with the output convention. If it's not the case for + * some plan types, then the API should be overridden. For example, a typical row-to-columnar + * transition is at the same time a query plan node that requires for row input however produces + * columnar output. + */ +trait GlutenPlan + extends SparkPlan + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33OrLater + with GlutenPlan.SupportsRowBasedCompatible + with ConventionReq.KnownChildConvention { + + final override val supportsColumnar: Boolean = { + batchType() != Convention.BatchType.None + } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } + + override def batchType(): Convention.BatchType + + override def rowType0(): Convention.RowType + + override def requiredChildConvention(): Seq[ConventionReq] = { + // In the normal case, children's convention should follow parent node's convention. + val childReq = Convention.of(rowType(), batchType()).asReq() + Seq.tabulate(children.size)( + _ => { + childReq + }) + } +} + +object GlutenPlan { + // To be compatible with Spark (version < 3.3) + trait SupportsRowBasedCompatible { + def supportsRowBased(): Boolean = { + throw new GlutenException("Illegal state: The method is not expected to be called") + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala index 3378344253dd..5b440302a09b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenColumnarRule.scala @@ -31,11 +31,9 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.vectorized.ColumnarBatch object GlutenColumnarRule { - // Utilities to infer columnar rule's caller's property: // ApplyColumnarRulesAndInsertTransitions#outputsColumnar. - - case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyRowOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = false override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -47,7 +45,7 @@ object GlutenColumnarRule { copy(child = newChild) } - case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { + private case class DummyColumnarOutputExec(override val child: SparkPlan) extends UnaryExecNode { override def supportsColumnar: Boolean = true override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = @@ -99,9 +97,8 @@ case class GlutenColumnarRule( "This should not happen. Please leave an issue at" + " https://github.com/apache/incubator-gluten.") } - val vanillaPlan = Transitions.insertTransitions(originalPlan, outputsColumnar) + val vanillaPlan = Transitions.insert(originalPlan, outputsColumnar) val applier = applierBuilder.apply(session) applier.apply(vanillaPlan, outputsColumnar) } - } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala index d5afc6b7e7b0..794f38365a0e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/GlutenSessionExtensions.scala @@ -17,8 +17,8 @@ package org.apache.gluten.extension import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.backend.Component +import org.apache.gluten.extension.injector.Injector import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSessionExtensions @@ -28,7 +28,7 @@ private[gluten] class GlutenSessionExtensions with Logging { import GlutenSessionExtensions._ override def apply(exts: SparkSessionExtensions): Unit = { - val injector = new RuleInjector(exts) + val injector = new Injector(exts) injector.control.disableOn { session => val glutenEnabledGlobally = session.conf @@ -47,7 +47,8 @@ private[gluten] class GlutenSessionExtensions logDebug(s"Gluten is disabled by variable: glutenEnabledForThread: $glutenEnabledForThread") disabled } - Backend.get().injectRules(injector) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(injector)) injector.inject() } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala index ecf13967e3ef..bf7b84c9b316 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/ColumnarRuleApplier.scala @@ -31,7 +31,7 @@ object ColumnarRuleApplier { val session: SparkSession, val ac: AdaptiveContext, val outputsColumnar: Boolean) { - val conf: GlutenConfig = { + val glutenConf: GlutenConfig = { new GlutenConfig(session.sessionState.conf) } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala new file mode 100644 index 000000000000..ea8f37823b3c --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/FallbackTag.scala @@ -0,0 +1,121 @@ +/* + * 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.gluten.extension.columnar + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} +import org.apache.spark.sql.execution.SparkPlan + +import org.apache.commons.lang3.exception.ExceptionUtils + +sealed trait FallbackTag { + val stacktrace: Option[String] = + if (FallbackTags.DEBUG) { + Some(ExceptionUtils.getStackTrace(new Throwable())) + } else None + + def reason(): String +} + +object FallbackTag { + + /** A tag that stores one reason text of fall back. */ + case class Appendable(override val reason: String) extends FallbackTag + + /** + * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is + * added to plan. + */ + case class Exclusive(override val reason: String) extends FallbackTag + + trait Converter[T] { + def from(obj: T): Option[FallbackTag] + } + + object Converter { + implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) + + implicit object FromString extends Converter[String] { + override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) + } + } +} + +object FallbackTags { + val TAG: TreeNodeTag[FallbackTag] = + TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") + + val DEBUG = false + + /** + * If true, the plan node will be guaranteed fallback to Vanilla plan node while being + * implemented. + * + * If false, the plan still has chance to be turned into "non-transformable" in any another + * validation rule. So user should not consider the plan "transformable" unless all validation + * rules are passed. + */ + def nonEmpty(plan: SparkPlan): Boolean = { + getOption(plan).nonEmpty + } + + /** + * If true, it implies the plan maybe transformable during validation phase but not guaranteed, + * since another validation rule could turn it to "non-transformable" before implementing the plan + * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan + * node while being implemented. + */ + def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) + + def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { + val tagOption = getOption(plan) + val newTagOption = converter.from(t) + + val mergedTagOption: Option[FallbackTag] = + (tagOption ++ newTagOption).reduceOption[FallbackTag] { + // New tag comes while the plan was already tagged, merge. + case (_, exclusive: FallbackTag.Exclusive) => + exclusive + case (exclusive: FallbackTag.Exclusive, _) => + exclusive + case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => + FallbackTag.Appendable(s"${l.reason}; ${r.reason}") + } + mergedTagOption + .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) + } + + def untag(plan: TreeNode[_]): Unit = { + plan.unsetTagValue(TAG) + } + + def get(plan: TreeNode[_]): FallbackTag = { + getOption(plan).getOrElse( + throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) + } + + def getOption(plan: TreeNode[_]): Option[FallbackTag] = { + plan.getTagValue(TAG) + } +} + +case class RemoveFallbackTagRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + plan.foreach(FallbackTags.untag) + plan + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala index 7ddeb33c7de5..04cd70656ef6 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedApplier.scala @@ -21,7 +21,6 @@ import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.util.AdaptiveContext import org.apache.gluten.logging.LogLevelUtil -import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule @@ -31,11 +30,9 @@ import org.apache.spark.sql.execution.SparkPlan * Columnar rule applier that optimizes, implements Spark plan into Gluten plan by enumerating on * all the possibilities of executable Gluten plans, then choose the best plan among them. * - * NOTE: This is still working in progress. We still have a bunch of heuristic rules in this - * implementation's rule list. Future work will include removing them from the list then - * implementing them in EnumeratedTransform. + * NOTE: We still have a bunch of heuristic rules in this implementation's rule list. Future work + * will include removing them from the list then implementing them in EnumeratedTransform. */ -@Experimental class EnumeratedApplier( session: SparkSession, ruleBuilders: Seq[ColumnarRuleCall => Rule[SparkPlan]]) @@ -64,3 +61,5 @@ class EnumeratedApplier( } } } + +object EnumeratedApplier {} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala new file mode 100644 index 000000000000..f1a325bc4379 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala @@ -0,0 +1,85 @@ +/* + * 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.gluten.extension.columnar.enumerated + +import org.apache.gluten.backend.Component +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.enumerated.planner.GlutenOptimization +import org.apache.gluten.extension.columnar.enumerated.planner.property.Conv +import org.apache.gluten.extension.injector.Injector +import org.apache.gluten.extension.util.AdaptiveContext +import org.apache.gluten.logging.LogLevelUtil +import org.apache.gluten.ras.CostModel +import org.apache.gluten.ras.property.PropertySet +import org.apache.gluten.ras.rule.RasRule + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution._ + +/** + * Rule to offload Spark query plan to Gluten query plan using a search algorithm and a defined cost + * model. + * + * The effect of this rule is similar to + * [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]], except that the 3 stages + * in the heuristic version, known as rewrite, validate, offload, will take place together + * individually for each Spark query plan node in RAS rule + * [[org.apache.gluten.extension.columnar.enumerated.RasOffload]]. + * + * The feature requires enabling RAS to function. + */ +case class EnumeratedTransform(costModel: CostModel[SparkPlan], rules: Seq[RasRule[SparkPlan]]) + extends Rule[SparkPlan] + with LogLevelUtil { + + private val optimization = { + GlutenOptimization + .builder() + .costModel(costModel) + .addRules(rules) + .create() + } + + private val convReq = Conv.any + + override def apply(plan: SparkPlan): SparkPlan = { + val constraintSet = PropertySet(Seq(convReq)) + val planner = optimization.newPlanner(plan, constraintSet) + val out = planner.plan() + out + } +} + +object EnumeratedTransform { + // Creates a static EnumeratedTransform rule for use in certain + // places that requires to emulate the offloading of a Spark query plan. + // + // TODO: Avoid using this and eventually remove the API. + def static(): EnumeratedTransform = { + val exts = new SparkSessionExtensions() + val dummyInjector = new Injector(exts) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(dummyInjector)) + val session = SparkSession.getActiveSession.getOrElse( + throw new GlutenException( + "HeuristicTransform#static can only be called when an active Spark session exists")) + val call = new ColumnarRuleCall(session, AdaptiveContext(session), false) + dummyInjector.gluten.ras.createEnumeratedTransform(call) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala similarity index 86% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala index 5b24f596395d..46bec0c59493 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/GlutenOptimization.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/GlutenOptimization.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner +package org.apache.gluten.extension.columnar.enumerated.planner -import org.apache.gluten.planner.metadata.GlutenMetadataModel -import org.apache.gluten.planner.plan.GlutenPlanModel -import org.apache.gluten.planner.property.GlutenPropertyModel +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel +import org.apache.gluten.extension.columnar.enumerated.planner.property.GlutenPropertyModel import org.apache.gluten.ras.{CostModel, Optimization, RasExplain} import org.apache.gluten.ras.rule.RasRule diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala similarity index 87% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala index 72c1cb8b430b..aa74f7736fbd 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCost.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCost.scala @@ -14,8 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.ras.Cost -case class GlutenCost(value: Long) extends Cost +case class LongCost(value: Long) extends Cost diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala new file mode 100644 index 000000000000..393ac35de42f --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCostModel.scala @@ -0,0 +1,124 @@ +/* + * 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.gluten.extension.columnar.enumerated.planner.cost + +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec +import org.apache.gluten.ras.{Cost, CostModel} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable + +abstract class LongCostModel extends CostModel[SparkPlan] { + private val infLongCost = Long.MaxValue + + override def costOf(node: SparkPlan): LongCost = node match { + case _: GroupLeafExec => throw new IllegalStateException() + case _ => LongCost(longCostOf(node)) + } + + private def longCostOf(node: SparkPlan): Long = node match { + case n => + val selfCost = selfLongCostOf(n) + + // Sum with ceil to avoid overflow. + def safeSum(a: Long, b: Long): Long = { + assert(a >= 0) + assert(b >= 0) + val sum = a + b + if (sum < a || sum < b) Long.MaxValue else sum + } + + (n.children.map(longCostOf).toList :+ selfCost).reduce(safeSum) + } + + def selfLongCostOf(node: SparkPlan): Long + + override def costComparator(): Ordering[Cost] = Ordering.Long.on { + case LongCost(value) => value + case _ => throw new IllegalStateException("Unexpected cost type") + } + + override def makeInfCost(): Cost = LongCost(infLongCost) +} + +object LongCostModel extends Logging { + def registry(): Registry = { + new Registry() + } + + /** + * Kind of a cost model. Output of #name() will be used as alias to identify the cost model + * instance from the registry. + */ + sealed trait Kind { + import Kind._ + all.synchronized { + val n = name() + if (all.contains(n)) { + throw new GlutenException(s"Cost mode kind $n already registered") + } + all += n -> this + } + + def name(): String + } + + object Kind { + private val all: mutable.Map[String, Kind] = mutable.Map() + def values(): Map[String, Kind] = all.toMap + } + + /** + * A cost model that is supposed to drive RAS planner create the same query plan with legacy + * planner. + */ + case object Legacy extends Kind { + override def name(): String = "legacy" + } + + /** A rough cost model with some empirical heuristics. */ + case object Rough extends Kind { + override def name(): String = "rough" + } + + /** Compared with rough, rough2 can be more precise to avoid the costly r2c. */ + case object Rough2 extends Kind { + override def name(): String = "rough2" + } + + class Registry private[LongCostModel] { + private val lookup: mutable.Map[Kind, LongCosterChain.Builder] = mutable.Map() + + // The registered coster will take lower precedence than all the existing + // registered costers in cost estimation. + def register(coster: LongCoster): Unit = synchronized { + val chainBuilder = builderOf(coster.kind()) + chainBuilder.register(coster) + } + + def get(kind: Kind): LongCostModel = synchronized { + builderOf(kind).build() + } + + private def builderOf(kind: Kind): LongCosterChain.Builder = { + lookup.getOrElseUpdate(kind, LongCosterChain.builder()) + } + } +} diff --git a/cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala similarity index 56% rename from cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala index 70aafdf07209..f06d1a4db829 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionLeast.cpp +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCoster.scala @@ -14,25 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include +package org.apache.gluten.extension.columnar.enumerated.planner.cost -namespace local_engine -{ -class SparkFunctionLeast : public FunctionGreatestestLeast -{ -public: - static constexpr auto name = "sparkLeast"; - static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } - SparkFunctionLeast() = default; - ~SparkFunctionLeast() override = default; - String getName() const override - { - return name; - } -}; +import org.apache.spark.sql.execution.SparkPlan -REGISTER_FUNCTION(SparkLeast) -{ - factory.registerFunction(); -} +/** + * Costs one single Spark plan node. The coster returns none if the input plan node is not + * recognizable. + * + * Used by the composite cost model [[LongCosterChain]]. + */ +trait LongCoster { + + /** The coster will be registered as part of the cost model associated with this kind. */ + def kind(): LongCostModel.Kind + + /** + * Calculates the long integer cost of the input query plan node. Note, this calculation should + * omit children's costs. + */ + def selfCostOf(node: SparkPlan): Option[Long] } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala new file mode 100644 index 000000000000..8b0c8b9f2d8a --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LongCosterChain.scala @@ -0,0 +1,64 @@ +/* + * 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.gluten.extension.columnar.enumerated.planner.cost +import org.apache.gluten.exception.GlutenException + +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable + +/** + * A [[LongCostModel]] implementation that consists of a set of sub-costers. + * + * The costers will apply in the same order they were registered or added. + */ +private class LongCosterChain private (costers: Seq[LongCoster]) extends LongCostModel { + override def selfLongCostOf(node: SparkPlan): Long = { + // Applies the costers respectively, returns when a coster gives a meaningful non-none number. + // If all costers give none, throw an error. + costers + .foldLeft[Option[Long]](None) { + case (None, coster) => + coster.selfCostOf(node) + case (c @ Some(_), _) => + c + } + .getOrElse(throw new GlutenException(s"Cost node found for node: $node")) + } +} + +object LongCosterChain { + def builder(): Builder = new Builder() + + class Builder private[LongCosterChain] { + private val costers = mutable.ListBuffer[LongCoster]() + private var out: Option[LongCosterChain] = None + + def register(coster: LongCoster): Builder = synchronized { + costers += coster + out = None + this + } + + private[cost] def build(): LongCosterChain = synchronized { + if (out.isEmpty) { + out = Some(new LongCosterChain(costers.toSeq)) + } + return out.get + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala similarity index 94% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala index f66c5290e95f..fabf6ac1fc79 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadata.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadata.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata import org.apache.gluten.ras.Metadata diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala index 7b95f1383d04..690964daa68d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/GlutenMetadataModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/GlutenMetadataModel.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.ras.{Metadata, MetadataModel} import org.apache.spark.internal.Logging diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala index 4c3bffd471ad..1886248e9f4e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/LogicalLink.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/LogicalLink.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.metadata.GlutenMetadataModel.Verifier +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel.Verifier import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala index 969d34d5cc82..1e8ff1ae59db 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/metadata/Schema.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/metadata/Schema.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.metadata +package org.apache.gluten.extension.columnar.enumerated.planner.metadata -import org.apache.gluten.planner.metadata.GlutenMetadataModel.Verifier +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadataModel.Verifier import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala similarity index 88% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala index abf7908a3872..94059020efca 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/plan/GlutenPlanModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/plan/GlutenPlanModel.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.plan +package org.apache.gluten.extension.columnar.enumerated.planner.plan +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.enumerated.planner.metadata.GlutenMetadata +import org.apache.gluten.extension.columnar.enumerated.planner.property.{Conv, ConvDef} import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} -import org.apache.gluten.extension.columnar.transition.Convention.{KnownBatchType, KnownRowType} -import org.apache.gluten.planner.metadata.GlutenMetadata -import org.apache.gluten.planner.property.{Conv, ConvDef} import org.apache.gluten.ras.{Metadata, PlanModel} import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.sql.shims.SparkShimLoader @@ -38,22 +38,20 @@ object GlutenPlanModel { PlanModelImpl } + // TODO: Make this inherit from GlutenPlan. case class GroupLeafExec( groupId: Int, metadata: GlutenMetadata, constraintSet: PropertySet[SparkPlan]) extends LeafExecNode - with KnownBatchType - with KnownRowType { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33OrLater + with GlutenPlan.SupportsRowBasedCompatible { private val req: Conv.Req = constraintSet.get(ConvDef).asInstanceOf[Conv.Req] override protected def doExecute(): RDD[InternalRow] = throw new IllegalStateException() override def output: Seq[Attribute] = metadata.schema().output - override def supportsColumnar: Boolean = { - batchType != Convention.BatchType.None - } - override val batchType: Convention.BatchType = { val out = req.req.requiredBatchType match { case ConventionReq.BatchType.Any => Convention.BatchType.None @@ -62,13 +60,21 @@ object GlutenPlanModel { out } - override val rowType: Convention.RowType = { + final override val supportsColumnar: Boolean = { + batchType != Convention.BatchType.None + } + + override val rowType0: Convention.RowType = { val out = req.req.requiredRowType match { case ConventionReq.RowType.Any => Convention.RowType.None case ConventionReq.RowType.Is(r) => r } out } + + final override val supportsRowBased: Boolean = { + rowType() != Convention.RowType.None + } } private object PlanModelImpl extends PlanModel[SparkPlan] { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala similarity index 78% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala index 78f290d19555..cfb32e76446a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/Conv.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/Conv.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.property +package org.apache.gluten.extension.columnar.enumerated.planner.property import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq, Transition} import org.apache.gluten.ras.{Property, PropertyDef} @@ -29,12 +29,24 @@ sealed trait Conv extends Property[SparkPlan] { } override def satisfies(other: Property[SparkPlan]): Boolean = { + // The following enforces strict type checking against `this` and `other` + // to make sure: + // + // 1. `this`, which came from user implementation of PropertyDef.getProperty, must be a `Prop` + // 2. `other` which came from user implementation of PropertyDef.getChildrenConstraints, + // must be a `Req` + // + // If the user implementation doesn't follow the criteria, cast error will be thrown. + // + // This can be a common practice to implement a safe Property for RAS. + // + // TODO: Add a similar case to RAS UTs. val req = other.asInstanceOf[Req] if (req.isAny) { return true } val prop = this.asInstanceOf[Prop] - val out = Transition.factory.satisfies(prop.prop, req.req) + val out = Transition.factory().satisfies(prop.prop, req.req) out } } @@ -52,7 +64,7 @@ object Conv { def findTransition(from: Conv, to: Conv): Transition = { val prop = from.asInstanceOf[Prop] val req = to.asInstanceOf[Req] - val out = Transition.factory.findTransition(prop.prop, req.req, new IllegalStateException()) + val out = Transition.factory().findTransition(prop.prop, req.req, new IllegalStateException()) out } @@ -80,7 +92,7 @@ object ConvDef extends PropertyDef[SparkPlan, Conv] { override def getChildrenConstraints( constraint: Property[SparkPlan], plan: SparkPlan): Seq[Conv] = { - val out = List.tabulate(plan.children.size)(_ => Conv.req(ConventionReq.get(plan))) + val out = ConventionReq.get(plan).map(Conv.req) out } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala similarity index 95% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala index 115ab4471fb6..bc7014f0fa1b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/property/GlutenPropertyModel.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/property/GlutenPropertyModel.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.property +package org.apache.gluten.extension.columnar.enumerated.planner.property import org.apache.gluten.ras._ diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala new file mode 100644 index 000000000000..b3458e1385c8 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/AddFallbackTags.scala @@ -0,0 +1,42 @@ +/* + * 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.gluten.extension.columnar.heuristic + +import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.validator.Validator + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +// Add fallback tags when validator returns negative outcome. +case class AddFallbackTags(validator: Validator) extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + plan.foreachUp { case p => addFallbackTag(p) } + plan + } + + private def addFallbackTag(plan: SparkPlan): Unit = { + val outcome = validator.validate(plan) + outcome match { + case Validator.Failed(reason) => + FallbackTags.add(plan, reason) + case Validator.Passed => + } + } +} + +object AddFallbackTags {} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.scala new file mode 100644 index 000000000000..e53c4cbf80e0 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/HeuristicTransform.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.gluten.extension.columnar.heuristic + +import org.apache.gluten.backend.Component +import org.apache.gluten.exception.GlutenException +import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode +import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode +import org.apache.gluten.extension.columnar.validator.Validator +import org.apache.gluten.extension.injector.Injector +import org.apache.gluten.extension.util.AdaptiveContext +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Rule to offload Spark query plan to Gluten query plan using programed heuristics. + * + * The procedure consists of 3 stages: + * + * 1. Rewrite 2. Validate 3. Offload + * + * In the rewrite stage, planner will try converting the Spark query plan to various forms of + * possible alternative Spark query plans, then choose the optimal one to send to next stage. During + * which, the same validation code that is about to be used in stage 2 might be invoked early to + * predict on the estimate "cost" of an alternative Spark query plan. + * + * Once the plan is rewritten, query planner will call native validation code in stage 2 to + * determine which part of the plan is offload-able or not, then add fallback tags to the + * non-offload-able parts. + * + * In stage 3, query planner will convert the offload-able Spark plan nodes into Gluten plan nodes. + */ +class HeuristicTransform private (all: Seq[Rule[SparkPlan]]) + extends Rule[SparkPlan] + with LogLevelUtil { + override def apply(plan: SparkPlan): SparkPlan = { + all.foldLeft(plan) { + case (plan, single) => + single(plan) + } + } +} + +object HeuristicTransform { + def withRules(all: Seq[Rule[SparkPlan]]): HeuristicTransform = { + new HeuristicTransform(all) + } + + case class Single( + validator: Validator, + rewriteRules: Seq[RewriteSingleNode], + offloadRules: Seq[OffloadSingleNode]) + extends Rule[SparkPlan] { + private val validate = AddFallbackTags(validator) + private val rewrite = RewriteSparkPlanRulesManager(validate, rewriteRules) + private val offload = LegacyOffload(offloadRules) + + override def apply(plan: SparkPlan): SparkPlan = { + Seq(rewrite, validate, offload).foldLeft(plan) { + case (plan, stage) => + stage(plan) + } + } + } + + // Creates a static HeuristicTransform rule for use in certain + // places that requires to emulate the offloading of a Spark query plan. + // + // TODO: Avoid using this and eventually remove the API. + def static(): HeuristicTransform = { + val exts = new SparkSessionExtensions() + val dummyInjector = new Injector(exts) + // Components should override Backend's rules. Hence, reversed injection order is applied. + Component.sorted().reverse.foreach(_.injectRules(dummyInjector)) + val session = SparkSession.getActiveSession.getOrElse( + throw new GlutenException( + "HeuristicTransform#static can only be called when an active Spark session exists")) + val call = new ColumnarRuleCall(session, AdaptiveContext(session), false) + dummyInjector.gluten.legacy.createHeuristicTransform(call) + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala new file mode 100644 index 000000000000..9249e6ebf742 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/LegacyOffload.scala @@ -0,0 +1,38 @@ +/* + * 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.gluten.extension.columnar.heuristic + +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode +import org.apache.gluten.logging.LogLevelUtil + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +class LegacyOffload(rules: Seq[OffloadSingleNode]) extends Rule[SparkPlan] with LogLevelUtil { + + def apply(plan: SparkPlan): SparkPlan = { + val out = + rules.foldLeft(plan)((p, rule) => p.transformUp { case p => rule.offload(p) }) + out + } +} + +object LegacyOffload { + def apply(rules: Seq[OffloadSingleNode]): LegacyOffload = { + new LegacyOffload(rules) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala similarity index 78% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala index c6d5e1cca1f7..59a370059015 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSparkPlanRulesManager.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/heuristic/RewriteSparkPlanRulesManager.scala @@ -14,21 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar.rewrite +package org.apache.gluten.extension.columnar.heuristic -import org.apache.gluten.extension.columnar.{AddFallbackTagRule, FallbackTag, FallbackTags} -import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} +import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.aggregate.BaseAggregateExec -import org.apache.spark.sql.execution.joins.BaseJoinExec -import org.apache.spark.sql.execution.python.ArrowEvalPythonExec -import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.execution.{LeafExecNode, ProjectExec, SparkPlan} case class RewrittenNodeWall(originalChild: SparkPlan) extends LeafExecNode { override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() @@ -45,31 +41,20 @@ case class RewrittenNodeWall(originalChild: SparkPlan) extends LeafExecNode { * * Note that, this rule does not touch and tag these operators who does not need to rewrite. */ -class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode]) +class RewriteSparkPlanRulesManager private ( + validateRule: Rule[SparkPlan], + rewriteRules: Seq[RewriteSingleNode]) extends Rule[SparkPlan] { private def mayNeedRewrite(plan: SparkPlan): Boolean = { - FallbackTags.maybeOffloadable(plan) && { - plan match { - case _: SortExec => true - case _: TakeOrderedAndProjectExec => true - case _: BaseAggregateExec => true - case _: BaseJoinExec => true - case _: WindowExec => true - case _: FilterExec => true - case _: FileSourceScanExec => true - case _: ExpandExec => true - case _: GenerateExec => true - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => true - case _: ArrowEvalPythonExec => true - case _ => false - } - } + FallbackTags.maybeOffloadable(plan) && rewriteRules.forall(_.isRewritable(plan)) } private def getFallbackTagBack(rewrittenPlan: SparkPlan): Option[FallbackTag] = { // The rewritten plan may contain more nodes than origin, for now it should only be // `ProjectExec`. + // TODO: Find a better approach than checking `p.isInstanceOf[ProjectExec]` which is not + // general. val target = rewrittenPlan.collect { case p if !p.isInstanceOf[ProjectExec] && !p.isInstanceOf[RewrittenNodeWall] => p } @@ -96,7 +81,6 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } override def apply(plan: SparkPlan): SparkPlan = { - val addHint = AddFallbackTagRule() plan.transformUp { case origin if mayNeedRewrite(origin) => // Add a wall to avoid transforming unnecessary nodes. @@ -114,7 +98,7 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] // We do not add tag and leave it to the outside `AddFallbackTagRule`. origin } else { - addHint.apply(rewrittenPlan) + validateRule.apply(rewrittenPlan) val tag = getFallbackTagBack(rewrittenPlan) if (tag.isDefined) { // If the rewritten plan is still not transformable, return the original plan. @@ -136,7 +120,9 @@ class RewriteSparkPlanRulesManager private (rewriteRules: Seq[RewriteSingleNode] } object RewriteSparkPlanRulesManager { - def apply(): Rule[SparkPlan] = { - new RewriteSparkPlanRulesManager(RewriteSingleNode.allRules()) + def apply( + validateRule: Rule[SparkPlan], + rewriteRules: Seq[RewriteSingleNode]): Rule[SparkPlan] = { + new RewriteSparkPlanRulesManager(validateRule, rewriteRules) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala similarity index 60% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala index 000dbac7ba47..232973f53a5d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNode.scala @@ -14,29 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar.validator +package org.apache.gluten.extension.columnar.offload +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan -trait Validator { - import Validator._ - def validate(plan: SparkPlan): OutCome - - final def pass(): OutCome = { - Passed - } - - final def fail(p: SparkPlan): OutCome = { - Validator.Failed(s"[${getClass.getSimpleName}] Validation failed on node ${p.nodeName}") - } - - final def fail(reason: String): OutCome = { - Validator.Failed(reason) - } -} - -object Validator { - sealed trait OutCome - case object Passed extends OutCome - case class Failed private (reason: String) extends OutCome +/** + * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed + * in native, and the internals of execution is subject by backend's implementation. + * + * Note: Only the current plan node is supposed to be open to modification. Do not access or modify + * the children node. Tree-walking is done by caller of this trait. + */ +trait OffloadSingleNode extends Logging { + def offload(plan: SparkPlan): SparkPlan } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala similarity index 89% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala index 551cfd599abd..38a8031a5cab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteSingleNode.scala @@ -30,11 +30,6 @@ import org.apache.spark.sql.execution.SparkPlan * TODO: Ideally for such API we'd better to allow multiple alternative outputs. */ trait RewriteSingleNode { + def isRewritable(plan: SparkPlan): Boolean def rewrite(plan: SparkPlan): SparkPlan } - -object RewriteSingleNode { - def allRules(): Seq[RewriteSingleNode] = { - Seq(RewriteIn, RewriteMultiChildrenCount, RewriteJoin, PullOutPreProject, PullOutPostProject) - } -} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala index 55bcb84d2b10..0e5387559674 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Convention.scala @@ -17,6 +17,11 @@ package org.apache.gluten.extension.columnar.transition import org.apache.spark.sql.execution.{ColumnarToRowExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.util.SparkVersionUtil + +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.mutable /** * Convention of a query plan consists of the row data type and columnar data type it supports to @@ -48,6 +53,19 @@ object Convention { } Convention.of(rowType(), batchType()) } + + def asReq(): ConventionReq = { + val rowTypeReq = conv.rowType match { + case Convention.RowType.None => ConventionReq.RowType.Any + case r => ConventionReq.RowType.Is(r) + } + + val batchTypeReq = conv.batchType match { + case Convention.BatchType.None => ConventionReq.BatchType.Any + case b => ConventionReq.BatchType.Is(b) + } + ConventionReq.of(rowTypeReq, batchTypeReq) + } } private case class Impl(override val rowType: RowType, override val batchType: BatchType) @@ -72,33 +90,62 @@ object Convention { } trait BatchType extends TransitionGraph.Vertex with Serializable { - Transition.graph.addVertex(this) + import BatchType._ + private val initialized: AtomicBoolean = new AtomicBoolean(false) + + final def ensureRegistered(): Unit = { + if (!initialized.compareAndSet(false, true)) { + // Already registered. + return + } + register() + } + + final private def register(): Unit = BatchType.synchronized { + assert(all.add(this)) + Transition.graph.addVertex(this) + registerTransitions() + } - final protected def fromRow(transition: Transition): Unit = { + ensureRegistered() + + /** + * User batch type could override this method to define transitions from/to this batch type by + * calling the subsequent protected APIs. + */ + protected[this] def registerTransitions(): Unit + + final protected[this] def fromRow(transition: Transition): Unit = { Transition.graph.addEdge(RowType.VanillaRow, this, transition) } - final protected def toRow(transition: Transition): Unit = { + final protected[this] def toRow(transition: Transition): Unit = { Transition.graph.addEdge(this, RowType.VanillaRow, transition) } - final protected def fromBatch(from: BatchType, transition: Transition): Unit = { + final protected[this] def fromBatch(from: BatchType, transition: Transition): Unit = { assert(from != this) Transition.graph.addEdge(from, this, transition) } - final protected def toBatch(to: BatchType, transition: Transition): Unit = { + final protected[this] def toBatch(to: BatchType, transition: Transition): Unit = { assert(to != this) Transition.graph.addEdge(this, to, transition) } } object BatchType { + private val all: mutable.Set[BatchType] = mutable.Set() + def values(): Set[BatchType] = all.toSet // None indicates that the plan doesn't support batch-based processing. - final case object None extends BatchType + final case object None extends BatchType { + override protected[this] def registerTransitions(): Unit = {} + } final case object VanillaBatch extends BatchType { - fromRow(RowToColumnarExec.apply) - toRow(ColumnarToRowExec.apply) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToColumnarExec.apply) + toRow(ColumnarToRowExec.apply) + } } } @@ -106,7 +153,37 @@ object Convention { def batchType(): BatchType } - trait KnownRowType { + sealed trait KnownRowType { def rowType(): RowType } + + trait KnownRowTypeForSpark33OrLater extends KnownRowType { + this: SparkPlan => + import KnownRowTypeForSpark33OrLater._ + + final override def rowType(): RowType = { + if (lteSpark32) { + // It's known that in Spark 3.2, one Spark plan node is considered either only having + // row-based support or only having columnar support at a time. + // Hence, if the plan supports columnar output, we'd disable its row-based support. + // The same for the opposite. + if (supportsColumnar) { + Convention.RowType.None + } else { + Convention.RowType.VanillaRow + } + } else { + rowType0() + } + } + + def rowType0(): RowType + } + + object KnownRowTypeForSpark33OrLater { + private val lteSpark32: Boolean = { + val v = SparkVersionUtil.majorMinorVersion() + SparkVersionUtil.compareMajorMinorVersion(v, (3, 2)) <= 0 + } + } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala index c3feefe9435a..bb894c2af0e1 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionFunc.scala @@ -16,26 +16,27 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.backend.Component +import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildConvention import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.execution.{ColumnarToRowExec, SparkPlan, UnionExec} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.QueryStageExec import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.util.SparkTestUtil /** ConventionFunc is a utility to derive [[Convention]] or [[ConventionReq]] from a query plan. */ sealed trait ConventionFunc { def conventionOf(plan: SparkPlan): Convention - def conventionReqOf(plan: SparkPlan): ConventionReq + def conventionReqOf(plan: SparkPlan): Seq[ConventionReq] } object ConventionFunc { trait Override { def rowTypeOf: PartialFunction[SparkPlan, Convention.RowType] = PartialFunction.empty def batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = PartialFunction.empty - def conventionReqOf: PartialFunction[SparkPlan, ConventionReq] = PartialFunction.empty + def conventionReqOf: PartialFunction[SparkPlan, Seq[ConventionReq]] = PartialFunction.empty } object Override { @@ -45,8 +46,9 @@ object ConventionFunc { // For testing, to make things work without a backend loaded. private var ignoreBackend: Boolean = false - // Visible for testing + // Visible for testing. def ignoreBackend[T](body: => T): T = synchronized { + assert(SparkTestUtil.isTesting) assert(!ignoreBackend) ignoreBackend = true try { @@ -68,11 +70,23 @@ object ConventionFunc { return Override.Empty } } - Backend.get().convFuncOverride() + // Components should override Backend's convention function. Hence, reversed injection order + // is applied. + val overrides = Component.sorted().reverse.map(_.convFuncOverride()) + new Override { + override val rowTypeOf: PartialFunction[SparkPlan, Convention.RowType] = { + overrides.map(_.rowTypeOf).reduce((l, r) => l.orElse(r)) + } + override val batchTypeOf: PartialFunction[SparkPlan, Convention.BatchType] = { + overrides.map(_.batchTypeOf).reduce((l, r) => l.orElse(r)) + } + override val conventionReqOf: PartialFunction[SparkPlan, Seq[ConventionReq]] = { + overrides.map(_.conventionReqOf).reduce((l, r) => l.orElse(r)) + } + } } private class BuiltinFunc(o: Override) extends ConventionFunc { - import BuiltinFunc._ override def conventionOf(plan: SparkPlan): Convention = { val conv = conventionOf0(plan) conv @@ -89,17 +103,6 @@ object ConventionFunc { } case q: QueryStageExec => conventionOf0(q.plan) case r: ReusedExchangeExec => conventionOf0(r.child) - case a: AdaptiveSparkPlanExec => - val rowType = rowTypeOf(a) - val batchType = if (a.supportsColumnar) { - // By default, we execute columnar AQE with backend batch output. - // See org.apache.gluten.extension.columnar.transition.InsertTransitions.apply - Backend.get().defaultBatchType - } else { - Convention.BatchType.None - } - val conv = Convention.of(rowType, batchType) - conv case other => val conv = Convention.of(rowTypeOf(other), batchTypeOf(other)) conv @@ -119,12 +122,24 @@ object ConventionFunc { case _ => Convention.RowType.None } - assert( - out == Convention.RowType.None || plan.isInstanceOf[Convention.KnownRowType] || - SparkShimLoader.getSparkShims.supportsRowBased(plan)) + checkRowType(plan, out) out } + private def checkRowType(plan: SparkPlan, rowType: Convention.RowType): Unit = { + if (SparkShimLoader.getSparkShims.supportsRowBased(plan)) { + assert( + rowType != Convention.RowType.None, + s"Plan ${plan.nodeName} supports row-based execution, " + + s"however #rowTypeOf returns None") + } else { + assert( + rowType == Convention.RowType.None, + s"Plan ${plan.nodeName} doesn't support row-based " + + s"execution, however #rowTypeOf returns $rowType") + } + } + private def batchTypeOf(plan: SparkPlan): Convention.BatchType = { val out = o.batchTypeOf.applyOrElse(plan, batchTypeOf0) out @@ -139,64 +154,63 @@ object ConventionFunc { case _ => Convention.BatchType.None } - assert(out == Convention.BatchType.None || plan.supportsColumnar) + checkBatchType(plan, out) out } - override def conventionReqOf(plan: SparkPlan): ConventionReq = { + private def checkBatchType(plan: SparkPlan, batchType: Convention.BatchType): Unit = { + if (plan.supportsColumnar) { + assert( + batchType != Convention.BatchType.None, + s"Plan ${plan.nodeName} supports columnar " + + s"execution, however #batchTypeOf returns None") + } else { + assert( + batchType == Convention.BatchType.None, + s"Plan ${plan.nodeName} doesn't support " + + s"columnar execution, however #batchTypeOf returns $batchType") + } + } + + override def conventionReqOf(plan: SparkPlan): Seq[ConventionReq] = { val req = o.conventionReqOf.applyOrElse(plan, conventionReqOf0) + assert(req.size == plan.children.size) req } - private def conventionReqOf0(plan: SparkPlan): ConventionReq = plan match { - case k: KnownChildrenConventions => - val reqs = k.requiredChildrenConventions().distinct - // This can be a temporary restriction. - assert( - reqs.size == 1, - "KnownChildrenConventions#requiredChildrenConventions should output the same element" + - " for all children") - reqs.head + private def conventionReqOf0(plan: SparkPlan): Seq[ConventionReq] = plan match { + case k: KnownChildConvention => + val reqs = k.requiredChildConvention() + reqs case RowToColumnarLike(_) => - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any) + Seq( + ConventionReq.of( + ConventionReq.RowType.Is(Convention.RowType.VanillaRow), + ConventionReq.BatchType.Any)) case ColumnarToRowExec(_) => - ConventionReq.of( - ConventionReq.RowType.Any, - ConventionReq.BatchType.Is(Convention.BatchType.VanillaBatch)) + Seq( + ConventionReq.of( + ConventionReq.RowType.Any, + ConventionReq.BatchType.Is(Convention.BatchType.VanillaBatch))) case write: DataWritingCommandExec if SparkShimLoader.getSparkShims.isPlannedV1Write(write) => // To align with ApplyColumnarRulesAndInsertTransitions#insertTransitions - ConventionReq.any + Seq(ConventionReq.any) case u: UnionExec => - // We force vanilla union to output row data to get best compatibility with vanilla Spark. + // We force vanilla union to output row data to get the best compatibility with vanilla + // Spark. // As a result it's a common practice to rewrite it with GlutenPlan for offloading. - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any) + Seq.tabulate(u.children.size)( + _ => + ConventionReq.of( + ConventionReq.RowType.Is(Convention.RowType.VanillaRow), + ConventionReq.BatchType.Any)) case other => // In the normal case, children's convention should follow parent node's convention. - // Note, we don't have consider C2R / R2C here since they are already removed by - // RemoveTransitions. - val thisConv = conventionOf0(other) - thisConv.asReq() - } - } - - private object BuiltinFunc { - implicit private class ConventionOps(conv: Convention) { - def asReq(): ConventionReq = { - val rowTypeReq = conv.rowType match { - case Convention.RowType.None => ConventionReq.RowType.Any - case r => ConventionReq.RowType.Is(r) - } - - val batchTypeReq = conv.batchType match { - case Convention.BatchType.None => ConventionReq.BatchType.Any - case b => ConventionReq.BatchType.Is(b) - } - ConventionReq.of(rowTypeReq, batchTypeReq) - } + val childReq = conventionOf0(other).asReq() + Seq.tabulate(other.children.size)( + _ => { + childReq + }) } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala index ce613bf7dbba..86637f2d5a7e 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/ConventionReq.scala @@ -16,8 +16,6 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend - import org.apache.spark.sql.execution.SparkPlan /** @@ -53,17 +51,16 @@ object ConventionReq { override val requiredBatchType: BatchType ) extends ConventionReq - val any: ConventionReq = Impl(RowType.Any, BatchType.Any) - val row: ConventionReq = Impl(RowType.Is(Convention.RowType.VanillaRow), BatchType.Any) - val vanillaBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Convention.BatchType.VanillaBatch)) - lazy val backendBatch: ConventionReq = - Impl(RowType.Any, BatchType.Is(Backend.get().defaultBatchType)) + val any: ConventionReq = of(RowType.Any, BatchType.Any) + val row: ConventionReq = ofRow(RowType.Is(Convention.RowType.VanillaRow)) + val vanillaBatch: ConventionReq = ofBatch(BatchType.Is(Convention.BatchType.VanillaBatch)) - def get(plan: SparkPlan): ConventionReq = ConventionFunc.create().conventionReqOf(plan) + def get(plan: SparkPlan): Seq[ConventionReq] = ConventionFunc.create().conventionReqOf(plan) def of(rowType: RowType, batchType: BatchType): ConventionReq = Impl(rowType, batchType) + def ofRow(rowType: RowType): ConventionReq = Impl(rowType, BatchType.Any) + def ofBatch(batchType: BatchType): ConventionReq = Impl(RowType.Any, batchType) - trait KnownChildrenConventions { - def requiredChildrenConventions(): Seq[ConventionReq] + trait KnownChildConvention { + def requiredChildConvention(): Seq[ConventionReq] } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala index ced9378ad69e..0a7f635b8bb0 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transition.scala @@ -101,22 +101,27 @@ object Transition { case (ConventionReq.RowType.Is(toRowType), ConventionReq.BatchType.Any) => from.rowType match { case Convention.RowType.None => + // Input query plan doesn't have recognizable row-based output, + // find columnar-to-row transition. graph.transitionOfOption(from.batchType, toRowType).getOrElse(orElse) - case fromRowType => + case fromRowType if toRowType == fromRowType => // We have only one single built-in row type. - assert(toRowType == fromRowType) Transition.empty + case _ => + throw new UnsupportedOperationException( + "Row-to-row transition is not yet supported") } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType)) => from.batchType match { case Convention.BatchType.None => + // Input query plan doesn't have recognizable columnar output, + // find row-to-columnar transition. graph.transitionOfOption(from.rowType, toBatchType).getOrElse(orElse) + case fromBatchType if toBatchType == fromBatchType => + Transition.empty case fromBatchType => - if (toBatchType == fromBatchType) { - Transition.empty - } else { - graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) - } + // Find columnar-to-columnar transition. + graph.transitionOfOption(fromBatchType, toBatchType).getOrElse(orElse) } case (ConventionReq.RowType.Any, ConventionReq.BatchType.Any) => Transition.empty diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala index 9987a65b0c4c..297485d84419 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/transition/Transitions.scala @@ -16,24 +16,22 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.backend.Backend +import org.apache.gluten.extension.columnar.transition.Convention.BatchType import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan import scala.annotation.tailrec -case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { +case class InsertTransitions(convReq: ConventionReq) extends Rule[SparkPlan] { private val convFunc = ConventionFunc.create() override def apply(plan: SparkPlan): SparkPlan = { // Remove all transitions at first. val removed = RemoveTransitions.apply(plan) val filled = fillWithTransitions(removed) - if (!outputsColumnar) { - return Transitions.toRowPlan(filled) - } - Transitions.toBackendBatchPlan(filled) + val out = Transitions.enforceReq(filled, convReq) + out } private def fillWithTransitions(plan: SparkPlan): SparkPlan = plan.transformUp { @@ -44,17 +42,17 @@ case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { if (node.children.isEmpty) { return node } - val convReq = convFunc.conventionReqOf(node) - val newChildren = node.children.map { - child => + val convReqs = convFunc.conventionReqOf(node) + val newChildren = node.children.zip(convReqs).map { + case (child, convReq) => val from = convFunc.conventionOf(child) if (from.isNone) { // For example, a union op with row child and columnar child at the same time, - // The plan is actually not executable and we cannot tell about its convention. + // The plan is actually not executable, and we cannot tell about its convention. child } else { val transition = - Transition.factory.findTransition(from, convReq, Transition.notFound(node)) + Transition.factory().findTransition(from, convReq, Transition.notFound(node)) val newChild = transition.apply(child) newChild } @@ -63,6 +61,17 @@ case class InsertTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { } } +object InsertTransitions { + def create(outputsColumnar: Boolean, batchType: BatchType): InsertTransitions = { + val conventionReq = if (outputsColumnar) { + ConventionReq.ofBatch(ConventionReq.BatchType.Is(batchType)) + } else { + ConventionReq.row + } + InsertTransitions(conventionReq) + } +} + object RemoveTransitions extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { case p => removeForNode(p) } @@ -76,42 +85,24 @@ object RemoveTransitions extends Rule[SparkPlan] { } object Transitions { - def insertTransitions(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { - InsertTransitions(outputsColumnar).apply(plan) + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertTransitions.create(outputsColumnar, BatchType.VanillaBatch).apply(plan) } def toRowPlan(plan: SparkPlan): SparkPlan = { - enforceReq( - plan, - ConventionReq.of( - ConventionReq.RowType.Is(Convention.RowType.VanillaRow), - ConventionReq.BatchType.Any)) - } - - def toBackendBatchPlan(plan: SparkPlan): SparkPlan = { - val backendBatchType = Backend.get().defaultBatchType - val out = toBatchPlan(plan, backendBatchType) - out - } - - def toVanillaBatchPlan(plan: SparkPlan): SparkPlan = { - val out = toBatchPlan(plan, Convention.BatchType.VanillaBatch) - out + enforceReq(plan, ConventionReq.row) } - private def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { - enforceReq( - plan, - ConventionReq.of(ConventionReq.RowType.Any, ConventionReq.BatchType.Is(toBatchType))) + def toBatchPlan(plan: SparkPlan, toBatchType: Convention.BatchType): SparkPlan = { + enforceReq(plan, ConventionReq.ofBatch(ConventionReq.BatchType.Is(toBatchType))) } - private def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { + def enforceReq(plan: SparkPlan, req: ConventionReq): SparkPlan = { val convFunc = ConventionFunc.create() val removed = RemoveTransitions.removeForNode(plan) - val transition = Transition.factory.findTransition( - convFunc.conventionOf(removed), - req, - Transition.notFound(removed, req)) + val transition = Transition + .factory() + .findTransition(convFunc.conventionOf(removed), req, Transition.notFound(removed, req)) val out = transition.apply(removed) out } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala new file mode 100644 index 000000000000..105dcb5db4aa --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/columnar/validator/Validator.scala @@ -0,0 +1,89 @@ +/* + * 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.gluten.extension.columnar.validator + +import org.apache.spark.sql.execution.SparkPlan + +import scala.collection.mutable +import scala.collection.mutable.ListBuffer + +trait Validator { + import Validator._ + def validate(plan: SparkPlan): OutCome + + final def pass(): OutCome = { + Passed + } + + final def fail(p: SparkPlan): OutCome = { + Validator.Failed(s"[${getClass.getSimpleName}] Validation failed on node ${p.nodeName}") + } + + final def fail(reason: String): OutCome = { + Validator.Failed(reason) + } +} + +object Validator { + sealed trait OutCome + case object Passed extends OutCome + case class Failed private (reason: String) extends OutCome + + def builder(): Builder = Builder() + + class Builder private { + private val buffer: ListBuffer[Validator] = mutable.ListBuffer() + + /** Add a custom validator to pipeline. */ + def add(validator: Validator): Builder = { + buffer += validator + this + } + + def build(): Validator = { + if (buffer.isEmpty) { + return NoopValidator + } + if (buffer.size == 1) { + return buffer.head + } + new ValidatorPipeline(buffer.toSeq) + } + + private class ValidatorPipeline(validators: Seq[Validator]) extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = { + val init: Validator.OutCome = pass() + val finalOut = validators.foldLeft(init) { + case (out, validator) => + out match { + case Validator.Passed => validator.validate(plan) + case Validator.Failed(_) => out + } + } + finalOut + } + } + + private object NoopValidator extends Validator { + override def validate(plan: SparkPlan): Validator.OutCome = pass() + } + } + + private object Builder { + def apply(): Builder = new Builder() + } +} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala index 498f040c9075..efe584d44126 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/GlutenInjector.scala @@ -20,12 +20,17 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.GlutenColumnarRule import org.apache.gluten.extension.columnar.ColumnarRuleApplier import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall -import org.apache.gluten.extension.columnar.enumerated.EnumeratedApplier -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier +import org.apache.gluten.extension.columnar.enumerated.{EnumeratedApplier, EnumeratedTransform} +import org.apache.gluten.extension.columnar.enumerated.planner.cost.{LongCoster, LongCostModel} +import org.apache.gluten.extension.columnar.heuristic.{HeuristicApplier, HeuristicTransform} +import org.apache.gluten.ras.CostModel +import org.apache.gluten.ras.rule.RasRule +import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SparkSessionExtensions} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.util.SparkReflectionUtil import scala.collection.mutable @@ -51,15 +56,25 @@ class GlutenInjector private[injector] (control: InjectorControl) { object GlutenInjector { class LegacyInjector { + private val preTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val transformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + private val postTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val fallbackPolicyBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val postBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] private val finalBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + def injectPreTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + preTransformBuilders += builder + } + def injectTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { transformBuilders += builder } + def injectPostTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + postTransformBuilders += builder + } + def injectFallbackPolicy(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { fallbackPolicyBuilders += builder } @@ -75,22 +90,77 @@ object GlutenInjector { private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = { new HeuristicApplier( session, - transformBuilders.toSeq, + (preTransformBuilders ++ Seq( + c => createHeuristicTransform(c)) ++ postTransformBuilders).toSeq, fallbackPolicyBuilders.toSeq, postBuilders.toSeq, - finalBuilders.toSeq) + finalBuilders.toSeq + ) + } + + def createHeuristicTransform(call: ColumnarRuleCall): HeuristicTransform = { + val all = transformBuilders.map(_(call)) + HeuristicTransform.withRules(all.toSeq) } } - class RasInjector { - private val ruleBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + class RasInjector extends Logging { + private val preTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + private val rasRuleBuilders = mutable.Buffer.empty[ColumnarRuleCall => RasRule[SparkPlan]] + private val costerBuilders = mutable.Buffer.empty[ColumnarRuleCall => LongCoster] + private val postTransformBuilders = mutable.Buffer.empty[ColumnarRuleCall => Rule[SparkPlan]] + + def injectPreTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + preTransformBuilders += builder + } + + def injectRasRule(builder: ColumnarRuleCall => RasRule[SparkPlan]): Unit = { + rasRuleBuilders += builder + } - def inject(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { - ruleBuilders += builder + def injectCoster(builder: ColumnarRuleCall => LongCoster): Unit = { + costerBuilders += builder + } + + def injectPostTransform(builder: ColumnarRuleCall => Rule[SparkPlan]): Unit = { + postTransformBuilders += builder } private[injector] def createApplier(session: SparkSession): ColumnarRuleApplier = { - new EnumeratedApplier(session, ruleBuilders.toSeq) + new EnumeratedApplier( + session, + (preTransformBuilders ++ Seq( + c => createEnumeratedTransform(c)) ++ postTransformBuilders).toSeq) + } + + def createEnumeratedTransform(call: ColumnarRuleCall): EnumeratedTransform = { + // Build RAS rules. + val rules = rasRuleBuilders.map(_(call)) + + // Build the cost model. + val costModelRegistry = LongCostModel.registry() + costerBuilders.foreach(cb => costModelRegistry.register(cb(call))) + val aliasOrClass = call.glutenConf.rasCostModel + val costModel = findCostModel(costModelRegistry, aliasOrClass) + + // Create transform. + EnumeratedTransform(costModel, rules.toSeq) + } + + private def findCostModel( + registry: LongCostModel.Registry, + aliasOrClass: String): CostModel[SparkPlan] = { + if (LongCostModel.Kind.values().contains(aliasOrClass)) { + val kind = LongCostModel.Kind.values()(aliasOrClass) + val model = registry.get(kind) + return model + } + val clazz = SparkReflectionUtil.classForName(aliasOrClass) + logInfo(s"Using user cost model: $aliasOrClass") + val ctor = clazz.getDeclaredConstructor() + ctor.setAccessible(true) + val model: CostModel[SparkPlan] = ctor.newInstance() + model } } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala similarity index 89% rename from gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala rename to gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala index c497a24a07b7..52a0c1580a7f 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/RuleInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/Injector.scala @@ -18,8 +18,8 @@ package org.apache.gluten.extension.injector import org.apache.spark.sql.SparkSessionExtensions -/** Injector used to inject query planner rules into Spark and Gluten. */ -class RuleInjector(extensions: SparkSessionExtensions) { +/** Injector used to inject extensible components into Spark and Gluten. */ +class Injector(extensions: SparkSessionExtensions) { val control = new InjectorControl() val spark: SparkInjector = new SparkInjector(control, extensions) val gluten: GlutenInjector = new GlutenInjector(control) @@ -30,5 +30,3 @@ class RuleInjector(extensions: SparkSessionExtensions) { gluten.inject(extensions) } } - -object RuleInjector {} diff --git a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala index 87942c4155e7..fe6db65c3a1d 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/extension/injector/SparkInjector.scala @@ -49,4 +49,8 @@ class SparkInjector private[injector] ( def injectFunction(functionDescription: FunctionDescription): Unit = { extensions.injectFunction(control.disabler().wrapFunction(functionDescription)) } + + def injectPreCBORule(builder: RuleBuilder): Unit = { + extensions.injectPreCBORule(control.disabler().wrapRule(builder)) + } } diff --git a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala index 2de1c7b4ed80..ef8ca2c974cb 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/iterator/Iterators.scala @@ -41,7 +41,7 @@ object Iterators { } def wrap[A](in: Iterator[A]): WrapperBuilder[A] = { - wrap(V1, in) + wrap(DEFAULT_VERSION, in) } def wrap[A](version: Version, in: Iterator[A]): WrapperBuilder[A] = { diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala new file mode 100644 index 000000000000..aa3f4e095861 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/GlutenShuffleManager.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.{ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.annotation.Experimental + +/** + * Shuffle manager that routes shuffle API calls to different shuffle managers registered by + * different backends. + * + * A SPIP may cause refactoring of this class in the future: + * https://issues.apache.org/jira/browse/SPARK-45792 + * + * Experimental: This is not expected to be used in production yet. Use backend shuffle manager + * (e.g., ColumnarShuffleManager or other RSS shuffle manager provided in Gluten's code + * base)instead. + */ +@Experimental +class GlutenShuffleManager(conf: SparkConf, isDriver: Boolean) extends ShuffleManager { + private val routerBuilder = ShuffleManagerRegistry.get().newRouterBuilder(conf, isDriver) + + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + routerBuilder.getOrBuild().registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + routerBuilder.getOrBuild().getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + routerBuilder + .getOrBuild() + .getReader(handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + routerBuilder.getOrBuild().unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + routerBuilder.getOrBuild().shuffleBlockResolver + } + + override def stop(): Unit = { + routerBuilder.getOrBuild().stop() + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala new file mode 100644 index 000000000000..502dd92efe80 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/LookupKey.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.ShuffleDependency + +/** + * Required during shuffle manager registration to determine whether the shuffle manager should be + * used for the particular shuffle dependency. + */ +trait LookupKey { + def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala new file mode 100644 index 000000000000..8b060c98184f --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerLookup.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.ShuffleDependency + +private class ShuffleManagerLookup(all: Seq[(LookupKey, ShuffleManager)]) { + private val allReversed = all.reverse + + def findShuffleManager[K, V, C](dependency: ShuffleDependency[K, V, C]): ShuffleManager = { + this.synchronized { + // The latest shuffle manager registered will be looked up earlier. + allReversed.find(_._1.accepts(dependency)).map(_._2).getOrElse { + throw new IllegalStateException(s"No ShuffleManager found for $dependency") + } + } + } + + def all(): Seq[ShuffleManager] = { + this.synchronized { + all.map(_._2) + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala new file mode 100644 index 000000000000..5b621a755d2b --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRegistry.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.SparkConf +import org.apache.spark.util.{SparkTestUtil, Utils} + +import scala.collection.mutable + +class ShuffleManagerRegistry private[ShuffleManagerRegistry] { + import ShuffleManagerRegistry._ + private val all: mutable.Buffer[(LookupKey, String)] = mutable.Buffer() + private val routerBuilders: mutable.Buffer[RouterBuilder] = mutable.Buffer() + private val classDeDup: mutable.Set[String] = mutable.Set() + + def register(lookupKey: LookupKey, shuffleManagerClass: String): Unit = { + val clazz = Utils.classForName(shuffleManagerClass) + require( + !clazz.isAssignableFrom(classOf[GlutenShuffleManager]), + "It's not allowed to register GlutenShuffleManager recursively") + require( + classOf[ShuffleManager].isAssignableFrom(clazz), + s"Shuffle manager class to register is not an implementation of Spark ShuffleManager: " + + s"$shuffleManagerClass" + ) + require( + !classDeDup.contains(shuffleManagerClass), + s"Shuffle manager class already registered: $shuffleManagerClass") + this.synchronized { + classDeDup += shuffleManagerClass + all += lookupKey -> shuffleManagerClass + // Invalidate all shuffle managers cached in each alive router builder instances. + // Then, once the router builder is accessed, a new router will be forced to create. + routerBuilders.foreach(_.invalidateCache()) + } + } + + // Visible for testing. + private[shuffle] def clear(): Unit = { + assert(SparkTestUtil.isTesting) + this.synchronized { + classDeDup.clear() + all.clear() + routerBuilders.foreach(_.invalidateCache()) + } + } + + private[shuffle] def newRouterBuilder(conf: SparkConf, isDriver: Boolean): RouterBuilder = + this.synchronized { + val out = new RouterBuilder(this, conf, isDriver) + routerBuilders += out + out + } +} + +object ShuffleManagerRegistry { + private val instance = new ShuffleManagerRegistry() + + def get(): ShuffleManagerRegistry = instance + + class RouterBuilder(registry: ShuffleManagerRegistry, conf: SparkConf, isDriver: Boolean) { + private var router: Option[ShuffleManagerRouter] = None + + private[ShuffleManagerRegistry] def invalidateCache(): Unit = synchronized { + router = None + } + + private[shuffle] def getOrBuild(): ShuffleManagerRouter = synchronized { + if (router.isEmpty) { + val instances = registry.all.map(key => key._1 -> instantiate(key._2, conf, isDriver)) + router = Some(new ShuffleManagerRouter(new ShuffleManagerLookup(instances.toSeq))) + } + router.get + } + + private def instantiate(clazz: String, conf: SparkConf, isDriver: Boolean): ShuffleManager = { + Utils + .instantiateSerializerOrShuffleManager[ShuffleManager](clazz, conf, isDriver) + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala new file mode 100644 index 000000000000..80aa9d804785 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/shuffle/ShuffleManagerRouter.scala @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle +import org.apache.spark.{ShuffleDependency, TaskContext} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.MergedBlockMeta +import org.apache.spark.storage.{BlockId, ShuffleBlockBatchId, ShuffleBlockId, ShuffleMergedBlockId} + +/** The internal shuffle manager instance used by GlutenShuffleManager. */ +private class ShuffleManagerRouter(lookup: ShuffleManagerLookup) extends ShuffleManager { + import ShuffleManagerRouter._ + private val cache = new Cache() + private val resolver = new BlockResolver(cache) + + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + val manager = lookup.findShuffleManager(dependency) + cache.store(shuffleId, manager).registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + cache.get(handle.shuffleId).getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + cache + .get(handle.shuffleId) + .getReader(handle, startMapIndex, endMapIndex, startPartition, endPartition, context, metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + cache.remove(shuffleId).unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = resolver + + override def stop(): Unit = { + assert(cache.size() == 0) + lookup.all().reverse.foreach(_.stop()) + } +} + +private object ShuffleManagerRouter { + private class Cache { + private val cache: java.util.Map[Int, ShuffleManager] = + new java.util.concurrent.ConcurrentHashMap() + + def store(shuffleId: Int, manager: ShuffleManager): ShuffleManager = { + cache.compute( + shuffleId, + (id, m) => { + assert(m == null, s"Shuffle manager was already cached for shuffle id: $id") + manager + }) + } + + def get(shuffleId: Int): ShuffleManager = { + val manager = cache.get(shuffleId) + assert(manager != null, s"Shuffle manager not registered for shuffle id: $shuffleId") + manager + } + + def remove(shuffleId: Int): ShuffleManager = { + val manager = cache.remove(shuffleId) + assert(manager != null, s"Shuffle manager not registered for shuffle id: $shuffleId") + manager + } + + def size(): Int = { + cache.size() + } + + def clear(): Unit = { + cache.clear() + } + } + + private class BlockResolver(cache: Cache) extends ShuffleBlockResolver { + override def getBlockData(blockId: BlockId, dirs: Option[Array[String]]): ManagedBuffer = { + val shuffleId = blockId match { + case id: ShuffleBlockId => + id.shuffleId + case batchId: ShuffleBlockBatchId => + batchId.shuffleId + case _ => + throw new IllegalArgumentException( + "GlutenShuffleManager: Unsupported shuffle block id: " + blockId) + } + cache.get(shuffleId).shuffleBlockResolver.getBlockData(blockId, dirs) + } + + override def getMergedBlockData( + blockId: ShuffleMergedBlockId, + dirs: Option[Array[String]]): Seq[ManagedBuffer] = { + val shuffleId = blockId.shuffleId + cache.get(shuffleId).shuffleBlockResolver.getMergedBlockData(blockId, dirs) + } + + override def getMergedBlockMeta( + blockId: ShuffleMergedBlockId, + dirs: Option[Array[String]]): MergedBlockMeta = { + val shuffleId = blockId.shuffleId + cache.get(shuffleId).shuffleBlockResolver.getMergedBlockMeta(blockId, dirs) + } + + override def stop(): Unit = { + throw new UnsupportedOperationException( + s"BlockResolver ${getClass.getSimpleName} doesn't need to be explicitly stopped") + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala index 40692346e052..ed243abb7f2b 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkReflectionUtil.scala @@ -20,4 +20,11 @@ object SparkReflectionUtil { def getSimpleClassName(cls: Class[_]): String = { Utils.getSimpleName(cls) } + + def classForName[C]( + className: String, + initialize: Boolean = true, + noSparkClassLoader: Boolean = false): Class[C] = { + Utils.classForName(className, initialize, noSparkClassLoader) + } } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala similarity index 72% rename from gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala rename to gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala index f674e7ec2e70..4fc09cf17c99 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/utils/ReflectionUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkTestUtil.scala @@ -14,15 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.utils +package org.apache.spark.util -import org.apache.spark.util.Utils - -object ReflectionUtil { - def classForName[C]( - className: String, - initialize: Boolean = true, - noSparkClassLoader: Boolean = false): Class[C] = { - Utils.classForName(className, initialize, noSparkClassLoader) +object SparkTestUtil { + def isTesting: Boolean = { + Utils.isTesting } } diff --git a/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.scala new file mode 100644 index 000000000000..6864d3caa031 --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/util/SparkVersionUtil.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.util + +object SparkVersionUtil { + def majorMinorVersion(): (Int, Int) = { + VersionUtils.majorMinorVersion(org.apache.spark.SPARK_VERSION) + } + + // Returns X. X < 0 if one < other, x == 0 if one == other, x > 0 if one > other. + def compareMajorMinorVersion(one: (Int, Int), other: (Int, Int)): Int = { + val base = 1000 + assert(one._2 < base && other._2 < base) + one._1 * base + one._2 - (other._1 * base + other._2) + } +} diff --git a/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala b/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala new file mode 100644 index 000000000000..a6f8bf2a0cbd --- /dev/null +++ b/gluten-core/src/test/scala/org/apache/gluten/backend/ComponentSuite.scala @@ -0,0 +1,100 @@ +/* + * 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.gluten.backend + +import org.apache.gluten.extension.injector.Injector + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.funsuite.AnyFunSuite + +class ComponentSuite extends AnyFunSuite with BeforeAndAfterAll { + import ComponentSuite._ + + private val d = new DummyComponentD() + d.ensureRegistered() + private val b = new DummyBackendB() + b.ensureRegistered() + private val a = new DummyBackendA() + a.ensureRegistered() + private val c = new DummyComponentC() + c.ensureRegistered() + private val e = new DummyComponentE() + e.ensureRegistered() + + test("Load order - sanity") { + val possibleOrders = + Set( + Seq(a, b, c, d, e), + Seq(a, b, d, c, e), + Seq(b, a, c, d, e), + Seq(b, a, d, c, e) + ) + + assert(possibleOrders.contains(Component.sorted())) + } + + test("Register again") { + assertThrows[IllegalArgumentException] { + new DummyBackendA().ensureRegistered() + } + } +} + +object ComponentSuite { + private class DummyBackendA extends Backend { + override def name(): String = "dummy-backend-a" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_BACKEND_A", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyBackendB extends Backend { + override def name(): String = "dummy-backend-b" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_BACKEND_B", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentC extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = classOf[DummyBackendA] :: Nil + + override def name(): String = "dummy-component-c" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_C", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentD extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = + Seq(classOf[DummyBackendA], classOf[DummyBackendB]) + + override def name(): String = "dummy-component-d" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_D", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } + + private class DummyComponentE extends Component { + override def dependencies(): Seq[Class[_ <: Component]] = + Seq(classOf[DummyBackendA], classOf[DummyComponentD]) + + override def name(): String = "dummy-component-e" + override def buildInfo(): Component.BuildInfo = + Component.BuildInfo("DUMMY_COMPONENT_E", "N/A", "N/A", "N/A") + override def injectRules(injector: Injector): Unit = {} + } +} diff --git a/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala b/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala new file mode 100644 index 000000000000..640fc0ab0796 --- /dev/null +++ b/gluten-core/src/test/scala/org/apache/spark/shuffle/GlutenShuffleManagerSuite.scala @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.shuffle + +import org.apache.spark.{Partitioner, ShuffleDependency, SparkConf, TaskContext} +import org.apache.spark.internal.config.SHUFFLE_MANAGER +import org.apache.spark.rdd.EmptyRDD +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.sql.test.SharedSparkSession + +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable + +class GlutenShuffleManagerSuite extends SharedSparkSession { + import GlutenShuffleManagerSuite._ + override protected def sparkConf: SparkConf = { + super.sparkConf + .set(SHUFFLE_MANAGER.key, classOf[GlutenShuffleManager].getName) + } + + override protected def afterEach(): Unit = { + val registry = ShuffleManagerRegistry.get() + registry.clear() + counter1.clear() + counter2.clear() + } + + test("register one") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + gm.stop() + gm.stop() + assert(counter1.count("stop") == 3) + } + + test("register two") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager2].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + // The statement calls #registerShuffle internally. + val dep = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep.shuffleId) + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 1) + + assert(counter1.count("stop") == 0) + assert(counter2.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + assert(counter2.count("stop") == 1) + } + + test("register two - disordered registration") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + val dep1 = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep1.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 0) + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager2].getName) + + // The statement calls #registerShuffle internally. + val dep2 = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep2.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 1) + + assert(counter1.count("stop") == 0) + assert(counter2.count("stop") == 0) + gm.stop() + assert(counter1.count("stop") == 1) + assert(counter2.count("stop") == 1) + } + + test("register two - with empty key") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = false + }, + classOf[ShuffleManager2].getName) + + val gm = spark.sparkContext.env.shuffleManager + assert(counter1.count("registerShuffle") == 0) + assert(counter2.count("registerShuffle") == 0) + // The statement calls #registerShuffle internally. + val dep = + new ShuffleDependency(new EmptyRDD[Product2[Any, Any]](spark.sparkContext), DummyPartitioner) + gm.unregisterShuffle(dep.shuffleId) + assert(counter1.count("registerShuffle") == 1) + assert(counter2.count("registerShuffle") == 0) + } + + test("register recursively") { + val registry = ShuffleManagerRegistry.get() + + assertThrows[IllegalArgumentException]( + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[GlutenShuffleManager].getName)) + } + + test("register duplicated") { + val registry = ShuffleManagerRegistry.get() + + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName) + assertThrows[IllegalArgumentException]( + registry.register( + new LookupKey { + override def accepts[K, V, C](dependency: ShuffleDependency[K, V, C]): Boolean = true + }, + classOf[ShuffleManager1].getName)) + } +} + +object GlutenShuffleManagerSuite { + private val counter1 = new InvocationCounter + private val counter2 = new InvocationCounter + + class ShuffleManager1(conf: SparkConf) extends ShuffleManager { + private val delegate = new SortShuffleManager(conf) + private val counter = counter1 + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + counter.increment("registerShuffle") + delegate.registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + counter.increment("getWriter") + delegate.getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + counter.increment("getReader") + delegate.getReader( + handle, + startMapIndex, + endMapIndex, + startPartition, + endPartition, + context, + metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + counter.increment("unregisterShuffle") + delegate.unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + counter.increment("shuffleBlockResolver") + delegate.shuffleBlockResolver + } + + override def stop(): Unit = { + counter.increment("stop") + delegate.stop() + } + } + + class ShuffleManager2(conf: SparkConf, isDriver: Boolean) extends ShuffleManager { + private val delegate = new SortShuffleManager(conf) + private val counter = counter2 + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + counter.increment("registerShuffle") + delegate.registerShuffle(shuffleId, dependency) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + counter.increment("getWriter") + delegate.getWriter(handle, mapId, context, metrics) + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + counter.increment("getReader") + delegate.getReader( + handle, + startMapIndex, + endMapIndex, + startPartition, + endPartition, + context, + metrics) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + counter.increment("unregisterShuffle") + delegate.unregisterShuffle(shuffleId) + } + + override def shuffleBlockResolver: ShuffleBlockResolver = { + counter.increment("shuffleBlockResolver") + delegate.shuffleBlockResolver + } + + override def stop(): Unit = { + counter.increment("stop") + delegate.stop() + } + } + + private class InvocationCounter { + private val counter: mutable.Map[String, AtomicInteger] = mutable.Map() + + def increment(name: String): Unit = synchronized { + counter.getOrElseUpdate(name, new AtomicInteger()).incrementAndGet() + } + + def count(name: String): Int = { + counter.getOrElse(name, new AtomicInteger()).get() + } + + def clear(): Unit = { + counter.clear() + } + } + + private object DummyPartitioner extends Partitioner { + override def numPartitions: Int = 0 + override def getPartition(key: Any): Int = 0 + } +} diff --git a/gluten-iceberg/pom.xml b/gluten-iceberg/pom.xml index 728924466002..bad2ec20e8e8 100644 --- a/gluten-iceberg/pom.xml +++ b/gluten-iceberg/pom.xml @@ -50,19 +50,6 @@ test-jar test - - org.apache.gluten - backends-velox - ${project.version} - test - - - org.apache.gluten - backends-velox - ${project.version} - test-jar - test - org.apache.spark spark-core_${scala.binary.version} diff --git a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala index 10d24c317cc1..1cbeb52a9213 100644 --- a/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala +++ b/gluten-iceberg/src/main/scala/org/apache/gluten/execution/IcebergScanTransformer.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration import org.apache.iceberg.spark.source.GlutenIcebergSourceUtil @@ -59,9 +58,7 @@ case class IcebergScanTransformer( override lazy val fileFormat: ReadFileFormat = GlutenIcebergSourceUtil.getFileFormat(scan) - override def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + override def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { val groupedPartitions = SparkShimLoader.getSparkShims.orderPartitions( scan, keyGroupedPartitioning, diff --git a/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala b/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala index ad8222cff5f5..a7451355b00f 100644 --- a/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala +++ b/gluten-iceberg/src/main/scala/org/apache/iceberg/spark/source/GlutenIcebergSourceUtil.scala @@ -16,6 +16,7 @@ */ package org.apache.iceberg.spark.source +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.substrait.rel.{IcebergLocalFilesBuilder, SplitInfo} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -25,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.types.StructType -import org.apache.iceberg.{CombinedScanTask, DeleteFile, FileFormat, FileScanTask, ScanTask, Schema} +import org.apache.iceberg._ import org.apache.iceberg.spark.SparkSchemaUtil import java.lang.{Long => JLong} @@ -50,8 +51,9 @@ object GlutenIcebergSourceUtil { val tasks = partition.taskGroup[ScanTask]().tasks().asScala asFileScanTask(tasks.toList).foreach { task => - val filePath = task.file().path().toString - paths.add(filePath) + paths.add( + BackendsApiManager.getTransformerApiInstance + .encodeFilePathIfNeed(task.file().path().toString)) starts.add(task.start()) lengths.add(task.length()) partitionColumns.add(getPartitionColumns(task, readPartitionSchema)) diff --git a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala similarity index 98% rename from gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala rename to gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala index 7f399ce629cf..f5bf24d1b64f 100644 --- a/gluten-iceberg/src/test/scala/org/apache/gluten/execution/VeloxIcebergSuite.scala +++ b/gluten-iceberg/src/test/scala/org/apache/gluten/execution/IcebergSuite.scala @@ -21,9 +21,10 @@ import org.apache.gluten.GlutenConfig import org.apache.spark.SparkConf import org.apache.spark.sql.Row -class VeloxIcebergSuite extends WholeStageTransformerSuite { - +abstract class IcebergSuite extends WholeStageTransformerSuite { protected val rootPath: String = getClass.getResource("/").getPath + // FIXME: This folder is in module backends-velox so is not accessible if profile backends-velox + // is not enabled during Maven build. override protected val resourcePath: String = "/tpch-data-parquet" override protected val fileFormat: String = "parquet" @@ -128,7 +129,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -208,7 +209,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 3) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 3) case _ => // do nothing } checkLengthAndPlan(df, 7) @@ -289,7 +290,7 @@ class VeloxIcebergSuite extends WholeStageTransformerSuite { case plan if plan.isInstanceOf[IcebergScanTransformer] => assert( plan.asInstanceOf[IcebergScanTransformer].getKeyGroupPartitioning.isDefined) - assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos(null).length == 1) + assert(plan.asInstanceOf[IcebergScanTransformer].getSplitInfos.length == 1) case _ => // do nothing } checkLengthAndPlan(df, 5) diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala index f705a2901be9..6cf15b0e8d6c 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/Ras.scala @@ -26,10 +26,7 @@ import scala.collection.mutable * https://github.com/apache/incubator-gluten/issues/5057. */ trait Optimization[T <: AnyRef] { - def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] + def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] def anyPropSet(): PropertySet[T] def withNewConfig(confFunc: RasConfig => RasConfig): Optimization[T] } @@ -47,10 +44,7 @@ object Optimization { implicit class OptimizationImplicits[T <: AnyRef](opt: Optimization[T]) { def newPlanner(plan: T): RasPlanner[T] = { - opt.newPlanner(plan, opt.anyPropSet(), List.empty) - } - def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { - opt.newPlanner(plan, constraintSet, List.empty) + opt.newPlanner(plan, opt.anyPropSet()) } } } @@ -113,11 +107,8 @@ class Ras[T <: AnyRef] private ( } } - override def newPlanner( - plan: T, - constraintSet: PropertySet[T], - altConstraintSets: Seq[PropertySet[T]]): RasPlanner[T] = { - RasPlanner(this, altConstraintSets, constraintSet, plan) + override def newPlanner(plan: T, constraintSet: PropertySet[T]): RasPlanner[T] = { + RasPlanner(this, constraintSet, plan) } override def anyPropSet(): PropertySet[T] = propertySetFactory().any() diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala index 327b980f38ec..626d2b1caf9c 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/RasPlanner.scala @@ -32,16 +32,12 @@ trait RasPlanner[T <: AnyRef] { } object RasPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { ras.config.plannerType match { case PlannerType.Exhaustive => - ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + ExhaustivePlanner(ras, constraintSet, plan) case PlannerType.Dp => - DpPlanner(ras, altConstraintSets, constraintSet, plan) + DpPlanner(ras, constraintSet, plan) } } } diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala index 3f2590dff8a4..2b601720bfd5 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/dp/DpPlanner.scala @@ -26,11 +26,7 @@ import org.apache.gluten.ras.property.PropertySet import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} // TODO: Branch and bound pruning. -private class DpPlanner[T <: AnyRef] private ( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T) +private class DpPlanner[T <: AnyRef] private (ras: Ras[T], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { import DpPlanner._ @@ -43,7 +39,6 @@ private class DpPlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId val memoTable = memo.table() val best = findBest(memoTable, groupId) @@ -70,12 +65,8 @@ private class DpPlanner[T <: AnyRef] private ( } object DpPlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new DpPlanner(ras, altConstraintSets: Seq[PropertySet[T]], constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new DpPlanner(ras, constraintSet, plan) } // Visited flag. diff --git a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala index c4d3e4881c43..f0cb42cf662a 100644 --- a/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala +++ b/gluten-ras/common/src/main/scala/org/apache/gluten/ras/exaustive/ExhaustivePlanner.scala @@ -27,7 +27,6 @@ import org.apache.gluten.ras.rule.{EnforcerRuleSet, RuleApplier, Shape} private class ExhaustivePlanner[T <: AnyRef] private ( ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], constraintSet: PropertySet[T], plan: T) extends RasPlanner[T] { @@ -40,7 +39,6 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } private lazy val best: (Best[T], KnownCostPath[T]) = { - altConstraintSets.foreach(propSet => memo.memorize(plan, propSet)) val groupId = rootGroupId explore() val memoState = memo.newState() @@ -72,12 +70,8 @@ private class ExhaustivePlanner[T <: AnyRef] private ( } object ExhaustivePlanner { - def apply[T <: AnyRef]( - ras: Ras[T], - altConstraintSets: Seq[PropertySet[T]], - constraintSet: PropertySet[T], - plan: T): RasPlanner[T] = { - new ExhaustivePlanner(ras, altConstraintSets, constraintSet, plan) + def apply[T <: AnyRef](ras: Ras[T], constraintSet: PropertySet[T], plan: T): RasPlanner[T] = { + new ExhaustivePlanner(ras, constraintSet, plan) } private class ExhaustiveExplorer[T <: AnyRef]( diff --git a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala index eb4babe069e5..06bb806f7d6b 100644 --- a/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala +++ b/gluten-ras/common/src/test/scala/org/apache/gluten/ras/PropertySuite.scala @@ -250,41 +250,6 @@ abstract class PropertySuite extends AnyFunSuite { assert(out == TypedUnary(TypeA, 8, PassNodeType(5, TypedLeaf(TypeA, 10)))) } - test(s"Property convert - (A, B), alternative conventions") { - object ConvertEnforcerAndTypeAToTypeB extends RasRule[TestNode] { - override def shift(node: TestNode): Iterable[TestNode] = node match { - case TypeEnforcer(TypeB, _, TypedBinary(TypeA, 5, left, right)) => - List(TypedBinary(TypeB, 0, left, right)) - case _ => List.empty - } - override def shape(): Shape[TestNode] = Shapes.fixedHeight(2) - } - - val ras = - Ras[TestNode]( - PlanModelImpl, - CostModelImpl, - MetadataModelImpl, - propertyModel(zeroDepth), - ExplainImpl, - RasRule.Factory.reuse(List(ConvertEnforcerAndTypeAToTypeB))) - .withNewConfig(_ => conf) - val plan = - TypedBinary(TypeA, 5, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10)), TypedLeaf(TypeA, 10)) - val planner = ras.newPlanner( - plan, - PropertySet(Seq(TypeAny)), - List(PropertySet(Seq(TypeB)), PropertySet(Seq(TypeC)))) - val out = planner.plan() - assert( - out == TypedBinary( - TypeB, - 0, - TypeEnforcer(TypeB, 1, TypedUnary(TypeA, 10, TypedLeaf(TypeA, 10))), - TypeEnforcer(TypeB, 1, TypedLeaf(TypeA, 10)))) - assert(planner.newState().memoState().allGroups().size == 9) - } - test(s"Property convert - (A, B), Unary only has TypeA") { object ReplaceNonUnaryByTypeBRule extends RasRule[TestNode] { override def shift(node: TestNode): Iterable[TestNode] = { diff --git a/gluten-substrait/pom.xml b/gluten-substrait/pom.xml index 14334c2443fb..e8eb6a57e6c4 100644 --- a/gluten-substrait/pom.xml +++ b/gluten-substrait/pom.xml @@ -117,60 +117,6 @@ 3.1.0.0-RC2 test - - org.seleniumhq.selenium - selenium-htmlunit-driver - 2.52.0 - test - - - io.trino.tpch - tpch - 1.1 - test - - - io.trino.tpcds - tpcds - 1.4 - test - - - org.kohsuke - github-api - 1.117 - test - - - io.jsonwebtoken - jjwt-api - 0.10.5 - test - - - io.jsonwebtoken - jjwt-impl - 0.10.5 - test - - - io.jsonwebtoken - jjwt-jackson - 0.10.5 - test - - - org.knowm.xchart - xchart - 3.6.5 - test - - - de.erichseifert.vectorgraphics2d - VectorGraphics2D - - - com.google.protobuf protobuf-java diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java index 04bb9d8cf400..9513f497602a 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/LocalFilesNode.java @@ -92,6 +92,11 @@ public List getPaths() { return paths; } + public void setPaths(List newPaths) { + paths.clear(); + paths.addAll(newPaths); + } + public void setFileSchema(StructType schema) { this.fileSchema = schema; } diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java index def1dca0a028..7d1931180847 100644 --- a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/RelBuilder.java @@ -27,6 +27,7 @@ import io.substrait.proto.CrossRel; import io.substrait.proto.JoinRel; +import io.substrait.proto.SetRel; import io.substrait.proto.SortField; import org.apache.spark.sql.catalyst.expressions.Attribute; @@ -317,4 +318,20 @@ public static RelNode makeGenerateRel( context.registerRelToOperator(operatorId); return new GenerateRelNode(input, generator, childOutput, extensionNode, outer); } + + public static RelNode makeSetRel( + List inputs, SetRel.SetOp setOp, SubstraitContext context, Long operatorId) { + context.registerRelToOperator(operatorId); + return new SetRelNode(inputs, setOp); + } + + public static RelNode makeSetRel( + List inputs, + SetRel.SetOp setOp, + AdvancedExtensionNode extensionNode, + SubstraitContext context, + Long operatorId) { + context.registerRelToOperator(operatorId); + return new SetRelNode(inputs, setOp, extensionNode); + } } diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java new file mode 100644 index 000000000000..ddcfb1701dd1 --- /dev/null +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/SetRelNode.java @@ -0,0 +1,62 @@ +/* + * 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.gluten.substrait.rel; + +import org.apache.gluten.substrait.extensions.AdvancedExtensionNode; + +import io.substrait.proto.Rel; +import io.substrait.proto.RelCommon; +import io.substrait.proto.SetRel; + +import java.io.Serializable; +import java.util.List; + +public class SetRelNode implements RelNode, Serializable { + private final List inputs; + private final SetRel.SetOp setOp; + private final AdvancedExtensionNode extensionNode; + + public SetRelNode(List inputs, SetRel.SetOp setOp, AdvancedExtensionNode extensionNode) { + this.inputs = inputs; + this.setOp = setOp; + this.extensionNode = extensionNode; + } + + public SetRelNode(List inputs, SetRel.SetOp setOp) { + this(inputs, setOp, null); + } + + @Override + public Rel toProtobuf() { + final RelCommon.Builder relCommonBuilder = RelCommon.newBuilder(); + relCommonBuilder.setDirect(RelCommon.Direct.newBuilder()); + final SetRel.Builder setBuilder = SetRel.newBuilder(); + setBuilder.setCommon(relCommonBuilder.build()); + if (inputs != null) { + for (RelNode input : inputs) { + setBuilder.addInputs(input.toProtobuf()); + } + } + setBuilder.setOp(setOp); + if (extensionNode != null) { + setBuilder.setAdvancedExtension(extensionNode.toProtobuf()); + } + final Rel.Builder builder = Rel.newBuilder(); + builder.setSet(setBuilder.build()); + return builder.build(); + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala index f1f46dd87e17..0d5b6b36da85 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendSettingsApi.scala @@ -18,22 +18,26 @@ package org.apache.gluten.backendsapi import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} import org.apache.spark.sql.types.StructField trait BackendSettingsApi { - def validateScan( + + /** The columnar-batch type this backend is by default using. */ + def primaryBatchType: Convention.BatchType + + def validateScanExec( format: ReadFileFormat, fields: Array[StructField], - rootPaths: Seq[String]): ValidationResult = ValidationResult.succeeded + rootPaths: Seq[String], + properties: Map[String, String]): ValidationResult = ValidationResult.succeeded def supportWriteFilesExec( format: FileFormat, @@ -75,12 +79,6 @@ trait BackendSettingsApi { def recreateJoinExecOnFallback(): Boolean = false - /** - * A shuffle key may be an expression. We would add a projection for this expression shuffle key - * and make it into a new column which the shuffle will refer to. But we need to remove it from - * the result columns from the shuffle. - */ - def supportShuffleWithProject(outputPartitioning: Partitioning, child: SparkPlan): Boolean = false def excludeScanExecFromCollapsedStage(): Boolean = false def rescaleDecimalArithmetic: Boolean = false @@ -123,6 +121,8 @@ trait BackendSettingsApi { def supportCartesianProductExec(): Boolean = false + def supportCartesianProductExecWithCondition(): Boolean = true + def supportBroadcastNestedLoopJoinExec(): Boolean = true def supportSampleExec(): Boolean = false diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala index 942058cc54d9..ab8ab3688916 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/BackendsApiManager.scala @@ -16,14 +16,16 @@ */ package org.apache.gluten.backendsapi -import org.apache.gluten.backend.Backend +import org.apache.gluten.backend.Component object BackendsApiManager { private lazy val backend: SubstraitBackend = initializeInternal() /** Initialize all backends api. */ private def initializeInternal(): SubstraitBackend = { - Backend.get().asInstanceOf[SubstraitBackend] + val loadedSubstraitBackends = Component.sorted().filter(_.isInstanceOf[SubstraitBackend]) + assert(loadedSubstraitBackends.size == 1, "More than one Substrait backends are loaded") + loadedSubstraitBackends.head.asInstanceOf[SubstraitBackend] } /** diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala index 69c9d37334de..11211bd0da91 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/IteratorApi.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.sql.utils.SparkInputMetricsUtil.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.SerializableConfiguration trait IteratorApi { @@ -38,8 +37,7 @@ trait IteratorApi { partitionSchema: StructType, fileFormat: ReadFileFormat, metadataColumnNames: Seq[String], - properties: Map[String, String], - serializableHadoopConf: SerializableConfiguration): SplitInfo + properties: Map[String, String]): SplitInfo /** Generate native row partition. */ def genPartitions( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala index 62008767f51b..453cfab4e487 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/MetricsApi.scala @@ -33,9 +33,14 @@ trait MetricsApi extends Serializable { "pipelineTime" -> SQLMetrics .createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) - def genInputIteratorTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] + def genInputIteratorTransformerMetrics( + child: SparkPlan, + sparkContext: SparkContext, + forBroadcast: Boolean): Map[String, SQLMetric] - def genInputIteratorTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genInputIteratorTransformerMetricsUpdater( + metrics: Map[String, SQLMetric], + forBroadcast: Boolean): MetricsUpdater def metricsUpdatingFunction( child: SparkPlan, @@ -121,6 +126,10 @@ trait MetricsApi extends Serializable { def genSampleTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genUnionTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] + + def genUnionTransformerMetricsUpdater(metrics: Map[String, SQLMetric]): MetricsUpdater + def genColumnarInMemoryTableMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala index 7c4c8577f421..745bbcb72d61 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/RuleApi.scala @@ -16,9 +16,9 @@ */ package org.apache.gluten.backendsapi -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector trait RuleApi { // Injects all Spark query planner rules used by the Gluten backend. - def injectRules(injector: RuleInjector): Unit + def injectRules(injector: Injector): Unit } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala index 667c0bdc25a9..ec032af92d96 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SparkPlanExecApi.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.python.ArrowEvalPythonExec import org.apache.spark.sql.execution.window._ -import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveUDFTransformer} +import org.apache.spark.sql.hive.HiveUDFTransformer import org.apache.spark.sql.types.{DecimalType, LongType, NullType, StructType} import org.apache.spark.sql.vectorized.ColumnarBatch @@ -64,9 +64,6 @@ trait SparkPlanExecApi { */ def genFilterExecTransformer(condition: Expression, child: SparkPlan): FilterExecTransformerBase - def genHiveTableScanExecTransformer(plan: SparkPlan): HiveTableScanExecTransformer = - HiveTableScanExecTransformer(plan) - def genProjectExecTransformer( projectList: Seq[NamedExpression], child: SparkPlan): ProjectExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala index d7785663d577..37be117105e2 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/SubstraitBackend.scala @@ -17,7 +17,7 @@ package org.apache.gluten.backendsapi import org.apache.gluten.backend.Backend -import org.apache.gluten.extension.injector.RuleInjector +import org.apache.gluten.extension.injector.Injector import org.apache.spark.SparkContext import org.apache.spark.api.plugin.PluginContext @@ -35,7 +35,7 @@ trait SubstraitBackend extends Backend { final override def onExecutorShutdown(): Unit = { listenerApi().onExecutorShutdown() } - final override def injectRules(injector: RuleInjector): Unit = { + final override def injectRules(injector: Injector): Unit = { ruleApi().injectRules(injector) } def iteratorApi(): IteratorApi diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala index 9642f63b3be3..69cea9c5470d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/backendsapi/TransformerApi.scala @@ -76,4 +76,7 @@ trait TransformerApi { def invalidateSQLExecutionResource(executionId: String): Unit = {} def genWriteParameters(fileFormat: FileFormat, writeOptions: Map[String, String]): Any + + /** use Hadoop Path class to encode the file path */ + def encodeFilePathIfNeed(filePath: String): String = filePath } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala index 09b56a5ac717..ac8e610956dc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicPhysicalOperatorTransformer.scala @@ -19,7 +19,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer} -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.`type`.TypeBuilder import org.apache.gluten.substrait.SubstraitContext @@ -61,8 +62,13 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP case _ => false } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = getRemainingCondition == null + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genFilterTransformerMetricsUpdater(metrics) + } def getRelNode( context: SubstraitContext, @@ -149,15 +155,15 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val remainingCondition = getRemainingCondition - val operatorId = context.nextOperatorId(this.nodeName) - if (remainingCondition == null) { + if (isNoop) { // The computing for this filter is not needed. - context.registerEmptyRelToOperator(operatorId) // Since some columns' nullability will be removed after this filter, we need to update the // outputAttributes of child context. - return TransformContext(childCtx.inputAttributes, output, childCtx.root) + return TransformContext(output, childCtx.root) } + + val operatorId = context.nextOperatorId(this.nodeName) + val remainingCondition = getRemainingCondition val currRel = getRelNode( context, remainingCondition, @@ -166,7 +172,7 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP childCtx.root, validation = false) assert(currRel != null, "Filter rel should be valid.") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } @@ -205,7 +211,7 @@ abstract class ProjectExecTransformerBase(val list: Seq[NamedExpression], val in val currRel = getRelNode(context, list, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Project Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override def output: Seq[Attribute] = list.map(_.toAttribute) @@ -255,15 +261,18 @@ abstract class ProjectExecTransformerBase(val list: Seq[NamedExpression], val in } } -// An alternatives for UnionExec. -case class ColumnarUnionExec(children: Seq[SparkPlan]) extends GlutenPlan { +// An alternative for UnionExec. +case class ColumnarUnionExec(children: Seq[SparkPlan]) extends ValidatablePlan { children.foreach { case w: WholeStageTransformer => + // FIXME: Avoid such practice for plan immutability. w.setOutputSchemaForPlan(output) case _ => } - override def supportsColumnar: Boolean = true + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None override def output: Seq[Attribute] = { children.map(_.output).transpose.map { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala index d7b824b397e5..73ed35e7190b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BasicScanExecTransformer.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.types.{BooleanType, StringType, StructField, StructType} -import org.apache.spark.util.SerializableConfiguration import com.google.protobuf.StringValue import io.substrait.proto.NamedStruct @@ -63,13 +62,11 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource def getProperties: Map[String, String] = Map.empty /** Returns the split infos that will be processed by the underlying native engine. */ - def getSplitInfos(serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { - getSplitInfosFromPartitions(getPartitions, serializableHadoopConf) + def getSplitInfos(): Seq[SplitInfo] = { + getSplitInfosFromPartitions(getPartitions) } - def getSplitInfosFromPartitions( - partitions: Seq[InputPartition], - serializableHadoopConf: SerializableConfiguration): Seq[SplitInfo] = { + def getSplitInfosFromPartitions(partitions: Seq[InputPartition]): Seq[SplitInfo] = { partitions.map( BackendsApiManager.getIteratorApiInstance .genSplitInfo( @@ -77,8 +74,7 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource getPartitionSchema, fileFormat, getMetadataColumns.map(_.name), - getProperties, - serializableHadoopConf)) + getProperties)) } override protected def doValidateInternal(): ValidationResult = { @@ -95,7 +91,7 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource } val validationResult = BackendsApiManager.getSettings - .validateScan(fileFormat, fields, getRootFilePaths) + .validateScanExec(fileFormat, fields, getRootFilePaths, getProperties) if (!validationResult.ok()) { return validationResult } @@ -162,6 +158,6 @@ trait BasicScanExecTransformer extends LeafTransformSupport with BaseDataSource extensionNode, context, context.nextOperatorId(this.nodeName)) - TransformContext(output, output, readNode) + TransformContext(output, readNode) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala index e1a1be8e29b5..d229117aa4e1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BatchScanExecTransformer.scala @@ -27,6 +27,7 @@ import org.apache.gluten.utils.FileIndexUtil import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} import org.apache.spark.sql.execution.datasources.v2.{BatchScanExecShim, FileScan} @@ -134,6 +135,10 @@ abstract class BatchScanExecTransformerBase( } override def doValidateInternal(): ValidationResult = { + if (!ScanTransformerFactory.supportedBatchScan(scan)) { + return ValidationResult.failed(s"Unsupported scan $scan") + } + if (pushedAggregate.nonEmpty) { return ValidationResult.failed(s"Unsupported aggregation push down for $scan.") } @@ -165,4 +170,13 @@ abstract class BatchScanExecTransformerBase( case "ClickHouseScan" => ReadFileFormat.MergeTreeReadFormat case _ => ReadFileFormat.UnknownFormat } + + override def simpleString(maxFields: Int): String = { + val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) + val runtimeFiltersString = s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" + val nativeFiltersString = s"NativeFilters: ${filterExprs().mkString("[", ",", "]")}" + val result = s"$nodeName$truncatedOutputString ${scan.description()}" + + s" $runtimeFiltersString $nativeFiltersString" + redact(result) + } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala index ae407b3b3efa..1e0c3d5a7e70 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/BroadcastNestedLoopJoinExecTransformer.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans.{FullOuter, InnerLike, JoinType, LeftExistence, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ExplainUtils, SparkPlan} import org.apache.spark.sql.execution.joins.BaseJoinExec import org.apache.spark.sql.execution.metric.SQLMetric @@ -45,6 +45,11 @@ abstract class BroadcastNestedLoopJoinExecTransformer( def joinBuildSide: BuildSide = buildSide + override def simpleStringWithNodeId(): String = { + val opId = ExplainUtils.getOpId(this) + s"$nodeName $joinType $joinBuildSide ($opId)".trim + } + override def leftKeys: Seq[Expression] = Nil override def rightKeys: Seq[Expression] = Nil diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala index 28bf1eeabd23..9e2f12bcf8ed 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/CartesianProductExecTransformer.scala @@ -18,7 +18,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ExpressionConverter -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.rel.RelBuilder @@ -45,7 +46,8 @@ import java.io.{IOException, ObjectOutputStream} */ case class ColumnarCartesianProductBridge(child: SparkPlan) extends UnaryExecNode with GlutenPlan { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() @@ -105,10 +107,18 @@ case class CartesianProductExecTransformer( context, operatorId ) - TransformContext(inputLeftOutput ++ inputRightOutput, output, currRel) + TransformContext(output, currRel) } override protected def doValidateInternal(): ValidationResult = { + if ( + !BackendsApiManager.getSettings.supportCartesianProductExecWithCondition() && + condition.nonEmpty + ) { + return ValidationResult.failed( + "CartesianProductExecTransformer with condition is not supported in this backend.") + } + if (!BackendsApiManager.getSettings.supportCartesianProductExec()) { return ValidationResult.failed("Cartesian product is not supported in this backend") } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala index f40a7f8f07f6..107bf544cd6f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarCoalesceExec.scala @@ -16,7 +16,8 @@ */ package org.apache.gluten.execution -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD @@ -28,9 +29,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode - with GlutenPlan { - - override def supportsColumnar: Boolean = true + with ValidatablePlan { override def output: Seq[Attribute] = child.output @@ -38,6 +37,10 @@ case class ColumnarCoalesceExec(numPartitions: Int, child: SparkPlan) if (numPartitions == 1) SinglePartition else UnknownPartitioning(numPartitions) } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala index fd86106bf367..5beaf49572ac 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ColumnarToRowExecBase.scala @@ -17,9 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -30,8 +28,7 @@ import org.apache.spark.sql.execution.{ColumnarToRowTransition, SparkPlan} abstract class ColumnarToRowExecBase(child: SparkPlan) extends ColumnarToRowTransition - with GlutenPlan - with KnownChildrenConventions { + with ValidatablePlan { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = @@ -43,6 +40,16 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def batchType(): Convention.BatchType = Convention.BatchType.None + + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + + override def requiredChildConvention(): Seq[ConventionReq] = { + List( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) + } + override def doExecuteBroadcast[T](): Broadcast[T] = { // Require for explicit implementation, otherwise throw error. super.doExecuteBroadcast[T]() @@ -53,9 +60,4 @@ abstract class ColumnarToRowExecBase(child: SparkPlan) override def doExecute(): RDD[InternalRow] = { doExecuteInternal() } - - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) - } - } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala index 63f76a25a231..c6936daaffe5 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ExpandExecTransformer.scala @@ -48,8 +48,13 @@ case class ExpandExecTransformer( AttributeSet.fromAttributeSets(projections.flatten.map(_.references)) } - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = projections == null || projections.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genExpandTransformerMetricsUpdater(metrics) + } // The GroupExpressions can output data with arbitrary partitioning, so set it // as UNKNOWN partitioning @@ -112,17 +117,16 @@ case class ExpandExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (projections == null || projections.isEmpty) { + if (isNoop) { // The computing for this Expand is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, projections, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Expand Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): ExpandExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala index d64c5ae016c5..7f3c6d4f9f47 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala @@ -26,6 +26,7 @@ import org.apache.gluten.utils.FileIndexUtil import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, PlanExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.FileSourceScanExecShim import org.apache.spark.sql.execution.datasources.HadoopFsRelation @@ -33,6 +34,8 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.util.collection.BitSet +import org.apache.commons.lang3.StringUtils + case class FileSourceScanExecTransformer( @transient override val relation: HadoopFsRelation, override val output: Seq[Attribute], @@ -157,10 +160,8 @@ abstract class FileSourceScanExecTransformerBase( override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genFileSourceScanTransformerMetricsUpdater(metrics) - override val nodeNamePrefix: String = "NativeFile" - override val nodeName: String = { - s"Scan $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" + s"ScanTransformer $relation ${tableIdentifier.map(_.unquotedString).getOrElse("")}" } override def getProperties: Map[String, String] = { @@ -192,6 +193,18 @@ abstract class FileSourceScanExecTransformerBase( case "CSVFileFormat" => ReadFileFormat.TextReadFormat case _ => ReadFileFormat.UnknownFormat } + + override def simpleString(maxFields: Int): String = { + val metadataEntries = metadata.toSeq.sorted.map { + case (key, value) => + key + ": " + StringUtils.abbreviate(redact(value), maxMetadataValueLength) + } + val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) + val nativeFiltersString = s"NativeFilters: ${filterExprs().mkString("[", ",", "]")}" + redact( + s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr" + + s" $nativeFiltersString") + } } object FileSourceScanExecTransformerBase { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala index af4a92f194c1..698d1f14c5b9 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/GenerateExecTransformerBase.scala @@ -79,7 +79,7 @@ abstract class GenerateExecTransformerBase( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) val relNode = getRelNode(context, childCtx.root, getGeneratorNode(context), validation = false) - TransformContext(child.output, output, relNode) + TransformContext(output, relNode) } protected def getExtensionNodeForValidation: AdvancedExtensionNode = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala index 12d08518509a..303c9e818f56 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/JoinUtils.scala @@ -265,6 +265,9 @@ object JoinUtils { case _: ExistenceJoin => inputBuildOutput.indices.map(ExpressionBuilder.makeSelection(_)) :+ ExpressionBuilder.makeSelection(buildOutput.size) + case LeftSemi | LeftAnti => + // When the left semi/anti join support the BuildLeft + leftOutput.indices.map(idx => ExpressionBuilder.makeSelection(idx + streamedOutput.size)) case LeftExistence(_) => leftOutput.indices.map(ExpressionBuilder.makeSelection(_)) case _ => @@ -312,7 +315,7 @@ object JoinUtils { } else { inputStreamedOutput ++ inputBuildOutput } - TransformContext(inputAttributes, output, rel) + TransformContext(output, rel) } def createCrossRel( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala similarity index 95% rename from gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala index 8859844be48e..0d49acd30f0d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/LimitExecTransformer.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.SparkPlan import scala.collection.JavaConverters._ -case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) +case class LimitExecTransformer(child: SparkPlan, offset: Long, count: Long) extends UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @@ -39,7 +39,7 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) override def output: Seq[Attribute] = child.output - override protected def withNewChildInternal(newChild: SparkPlan): LimitTransformer = + override protected def withNewChildInternal(newChild: SparkPlan): LimitExecTransformer = copy(child = newChild) override def metricsUpdater(): MetricsUpdater = @@ -57,7 +57,7 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) val childCtx = child.asInstanceOf[TransformSupport].transform(context) val operatorId = context.nextOperatorId(this.nodeName) val relNode = getRelNode(context, operatorId, offset, count, child.output, childCtx.root, false) - TransformContext(child.output, child.output, relNode) + TransformContext(child.output, relNode) } def getRelNode( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala index f4dd160b58b4..9f36b8fc672d 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/RowToColumnarExecBase.scala @@ -17,7 +17,7 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.spark.broadcast import org.apache.spark.rdd.RDD @@ -45,6 +45,14 @@ abstract class RowToColumnarExecBase(child: SparkPlan) final override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + + override def requiredChildConvention(): Seq[ConventionReq] = { + Seq(ConventionReq.row) + } + final override def doExecute(): RDD[InternalRow] = { child.execute() } @@ -54,8 +62,6 @@ abstract class RowToColumnarExecBase(child: SparkPlan) super.doExecuteBroadcast[T]() } - final override def supportsColumnar: Boolean = true - def doExecuteColumnarInternal(): RDD[ColumnarBatch] override def doExecuteColumnar(): RDD[ColumnarBatch] = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala index bed59b913a1e..8e664a3b6ebc 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SampleExecTransformer.scala @@ -118,7 +118,7 @@ case class SampleExecTransformer( val currRel = getRelNode(context, condition, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Filter rel should be valid.") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): SampleExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala index 52dad6da3773..dfdf2d2f34e1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/ScanTransformerFactory.scala @@ -16,12 +16,10 @@ */ package org.apache.gluten.execution -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.connector.read.Scan -import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import java.util.ServiceLoader @@ -58,8 +56,7 @@ object ScanTransformerFactory { } } - private def lookupBatchScanTransformer( - batchScanExec: BatchScanExec): BatchScanExecTransformerBase = { + def createBatchScanTransformer(batchScanExec: BatchScanExec): BatchScanExecTransformerBase = { val scan = batchScanExec.scan lookupDataSourceScanTransformer(scan.getClass.getName) match { case Some(clz) => @@ -69,46 +66,16 @@ object ScanTransformerFactory { .asInstanceOf[DataSourceScanTransformerRegister] .createDataSourceV2Transformer(batchScanExec) case _ => - scan match { - case _: FileScan => - BatchScanExecTransformer( - batchScanExec.output, - batchScanExec.scan, - batchScanExec.runtimeFilters, - table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScanExec) - ) - case _ => - throw new GlutenNotSupportException(s"Unsupported scan $scan") - } - } - } - - def createBatchScanTransformer( - batchScan: BatchScanExec, - validation: Boolean = false): SparkPlan = { - if (supportedBatchScan(batchScan.scan)) { - val transformer = lookupBatchScanTransformer(batchScan) - if (!validation) { - val validationResult = transformer.doValidate() - if (validationResult.ok()) { - transformer - } else { - FallbackTags.add(batchScan, validationResult.reason()) - batchScan - } - } else { - transformer - } - } else { - if (validation) { - throw new GlutenNotSupportException(s"Unsupported scan ${batchScan.scan}") - } - FallbackTags.add(batchScan, "The scan in BatchScanExec is not supported.") - batchScan + BatchScanExecTransformer( + batchScanExec.output, + batchScanExec.scan, + batchScanExec.runtimeFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScanExec) + ) } } - private def supportedBatchScan(scan: Scan): Boolean = scan match { + def supportedBatchScan(scan: Scan): Boolean = scan match { case _: FileScan => true case _ => lookupDataSourceScanTransformer(scan.getClass.getName).nonEmpty } @@ -132,5 +99,4 @@ object ScanTransformerFactory { ) Option(clz) } - } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala index b69925d60fd2..6f9564e6d54f 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/SortExecTransformer.scala @@ -44,8 +44,13 @@ case class SortExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genSortTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = sortOrder == null || sortOrder.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genSortTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output @@ -103,17 +108,16 @@ case class SortExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (sortOrder == null || sortOrder.isEmpty) { + if (isNoop) { // The computing for this project is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getRelNode(context, sortOrder, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Sort Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): SortExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala index b31471e21397..f19960ec1ca3 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/TakeOrderedAndProjectExecTransformer.scala @@ -17,7 +17,8 @@ package org.apache.gluten.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -40,10 +41,11 @@ case class TakeOrderedAndProjectExecTransformer( child: SparkPlan, offset: Int = 0) extends UnaryExecNode - with GlutenPlan { + with ValidatablePlan { override def outputPartitioning: Partitioning = SinglePartition override def outputOrdering: Seq[SortOrder] = sortOrder - override def supportsColumnar: Boolean = true + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override def output: Seq[Attribute] = { projectList.map(_.toAttribute) @@ -73,7 +75,7 @@ case class TakeOrderedAndProjectExecTransformer( var tagged: ValidationResult = null val orderingSatisfies = SortOrder.orderingSatisfies(child.outputOrdering, sortOrder) if (orderingSatisfies) { - val limitPlan = LimitTransformer(child, offset, limit) + val limitPlan = LimitExecTransformer(child, offset, limit) tagged = limitPlan.doValidate() } else { // Here we are validating sort + limit which is a kind of whole stage transformer, @@ -86,7 +88,7 @@ case class TakeOrderedAndProjectExecTransformer( if (!sortValidation.ok()) { return sortValidation } - val limitPlan = LimitTransformer(sortPlan, offset, limit) + val limitPlan = LimitExecTransformer(sortPlan, offset, limit) tagged = limitPlan.doValidate() } @@ -127,13 +129,13 @@ case class TakeOrderedAndProjectExecTransformer( // remove this WholeStageTransformer, put the new sort, limit and project // into a new whole stage. val localSortPlan = withLocalSort(wholeStage.child) - LimitTransformer(localSortPlan, limitBeforeShuffleOffset, limit) + LimitExecTransformer(localSortPlan, limitBeforeShuffleOffset, limit) case other => // if the child it is not WholeStageTransformer, add the adapter first // so that, later we can wrap WholeStageTransformer. val localSortPlan = withLocalSort( ColumnarCollapseTransformStages.wrapInputIteratorTransformer(other)) - LimitTransformer(localSortPlan, limitBeforeShuffleOffset, limit) + LimitExecTransformer(localSortPlan, limitBeforeShuffleOffset, limit) } val transformStageCounter: AtomicInteger = ColumnarCollapseTransformStages.transformStageCounter @@ -150,7 +152,7 @@ case class TakeOrderedAndProjectExecTransformer( sortOrder, false, ColumnarCollapseTransformStages.wrapInputIteratorTransformer(transformedShuffleExec)) - LimitTransformer(localSortPlan, offset, limit) + LimitExecTransformer(localSortPlan, offset, limit) } val projectPlan = if (projectList != child.output) { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.scala new file mode 100644 index 000000000000..d27558746a40 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/UnionExecTransformer.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.gluten.execution + +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.expression.ConverterUtils +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.metrics.MetricsUpdater +import org.apache.gluten.substrait.`type`.TypeBuilder +import org.apache.gluten.substrait.SubstraitContext +import org.apache.gluten.substrait.extensions.ExtensionBuilder +import org.apache.gluten.substrait.rel.{RelBuilder, RelNode} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.{SparkPlan, UnionExec} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +import io.substrait.proto.SetRel.SetOp + +import scala.collection.JavaConverters._ + +/** Transformer for UnionExec. Note: Spark's UnionExec represents a SQL UNION ALL. */ +case class UnionExecTransformer(children: Seq[SparkPlan]) extends TransformSupport { + private val union = UnionExec(children) + + // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. + @transient override lazy val metrics: Map[String, SQLMetric] = + BackendsApiManager.getMetricsApiInstance.genUnionTransformerMetrics(sparkContext) + + override def output: Seq[Attribute] = union.output + + override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = children.flatMap(getColumnarInputRDDs) + + override def metricsUpdater(): MetricsUpdater = + BackendsApiManager.getMetricsApiInstance.genUnionTransformerMetricsUpdater(metrics) + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[SparkPlan]): SparkPlan = + copy(children = newChildren) + + override protected def doValidateInternal(): ValidationResult = { + val context = new SubstraitContext + val operatorId = context.nextOperatorId(this.nodeName) + val relNode = getRelNode(context, operatorId, children.map(_.output), null, true) + doNativeValidation(context, relNode) + } + + override protected def doTransform(context: SubstraitContext): TransformContext = { + val childrenCtx = children.map(_.asInstanceOf[TransformSupport].transform(context)) + val operatorId = context.nextOperatorId(this.nodeName) + val relNode = + getRelNode(context, operatorId, children.map(_.output), childrenCtx.map(_.root), false) + TransformContext(output, relNode) + } + + private def getRelNode( + context: SubstraitContext, + operatorId: Long, + inputAttributes: Seq[Seq[Attribute]], + inputs: Seq[RelNode], + validation: Boolean): RelNode = { + if (validation) { + // Use the second level of nesting to represent N way inputs. + val inputTypeNodes = + inputAttributes.map( + attributes => + attributes.map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)).asJava) + val extensionNode = ExtensionBuilder.makeAdvancedExtension( + BackendsApiManager.getTransformerApiInstance.packPBMessage( + TypeBuilder + .makeStruct( + false, + inputTypeNodes.map(nodes => TypeBuilder.makeStruct(false, nodes)).asJava) + .toProtobuf)) + return RelBuilder.makeSetRel( + inputs.asJava, + SetOp.SET_OP_UNION_ALL, + extensionNode, + context, + operatorId) + } + RelBuilder.makeSetRel(inputs.asJava, SetOp.SET_OP_UNION_ALL, context, operatorId) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala index e1dfd3f5704a..dbfc11c136db 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WholeStageTransformer.scala @@ -18,14 +18,17 @@ package org.apache.gluten.execution import org.apache.gluten.{GlutenConfig, GlutenNumaBindingInfo} import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenException +import org.apache.gluten.exception.{GlutenException, GlutenNotSupportException} import org.apache.gluten.expression._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.metrics.{GlutenTimeMetric, MetricsUpdater} import org.apache.gluten.substrait.`type`.{TypeBuilder, TypeNode} import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.plan.{PlanBuilder, PlanNode} -import org.apache.gluten.substrait.rel.{RelNode, SplitInfo} +import org.apache.gluten.substrait.rel.{LocalFilesNode, RelNode, SplitInfo} +import org.apache.gluten.test.TestStats import org.apache.gluten.utils.SubstraitPlanPrinterUtil import org.apache.spark._ @@ -43,26 +46,90 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.SerializableConfiguration import com.google.common.collect.Lists +import org.apache.hadoop.fs.{FileSystem, Path} +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -case class TransformContext( - inputAttributes: Seq[Attribute], - outputAttributes: Seq[Attribute], - root: RelNode) +case class TransformContext(outputAttributes: Seq[Attribute], root: RelNode) case class WholeStageTransformContext(root: PlanNode, substraitContext: SubstraitContext = null) -trait TransformSupport extends GlutenPlan { +/** + * Base interface for a Gluten query plan that is also open to validation calls. + * + * Since https://github.com/apache/incubator-gluten/pull/2185. + */ +trait ValidatablePlan extends GlutenPlan with LogLevelUtil { + protected def glutenConf: GlutenConfig = GlutenConfig.getConf + + protected lazy val enableNativeValidation = glutenConf.enableNativeValidation + + /** + * Validate whether this SparkPlan supports to be transformed into substrait node in Native Code. + */ + final def doValidate(): ValidationResult = { + val schemaValidationResult = BackendsApiManager.getValidatorApiInstance + .doSchemaValidate(schema) + .map { + reason => + ValidationResult.failed(s"Found schema check failure for $schema, due to: $reason") + } + .getOrElse(ValidationResult.succeeded) + if (!schemaValidationResult.ok()) { + TestStats.addFallBackClassName(this.getClass.toString) + return schemaValidationResult + } + try { + TransformerState.enterValidation + val res = doValidateInternal() + if (!res.ok()) { + TestStats.addFallBackClassName(this.getClass.toString) + } + res + } catch { + case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => + if (!e.isInstanceOf[GlutenNotSupportException]) { + logDebug(s"Just a warning. This exception perhaps needs to be fixed.", e) + } + // FIXME: Use a validation-specific method to catch validation failures + TestStats.addFallBackClassName(this.getClass.toString) + logValidationMessage( + s"Validation failed with exception for plan: $nodeName, due to: ${e.getMessage}", + e) + ValidationResult.failed(e.getMessage) + } finally { + TransformerState.finishValidation + } + } + + protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded + + private def logValidationMessage(msg: => String, e: Throwable): Unit = { + if (glutenConf.printStackOnValidationFailure) { + logOnLevel(glutenConf.validationLogLevel, msg, e) + } else { + logOnLevel(glutenConf.validationLogLevel, msg) + } + } +} + +/** Base interface for a query plan that can be interpreted to Substrait representation. */ +trait TransformSupport extends ValidatablePlan { + override def batchType(): Convention.BatchType = { + BackendsApiManager.getSettings.primaryBatchType + } + + override def rowType0(): Convention.RowType = { + Convention.RowType.None + } final override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( s"${this.getClass.getSimpleName} doesn't support doExecute") } - final override lazy val supportsColumnar: Boolean = true - /** * Returns all the RDDs of ColumnarBatch which generates the input rows. * @@ -71,6 +138,17 @@ trait TransformSupport extends GlutenPlan { */ def columnarInputRDDs: Seq[RDD[ColumnarBatch]] + // Since https://github.com/apache/incubator-gluten/pull/2185. + protected def doNativeValidation(context: SubstraitContext, node: RelNode): ValidationResult = { + if (node != null && glutenConf.enableNativeValidation) { + val planNode = PlanBuilder.makePlan(context, Lists.newArrayList(node)) + BackendsApiManager.getValidatorApiInstance + .doNativeValidateWithFailureReason(planNode) + } else { + ValidationResult.succeeded + } + } + final def transform(context: SubstraitContext): TransformContext = { if (isCanonicalizedPlan) { throw new IllegalStateException( @@ -101,6 +179,9 @@ trait TransformSupport extends GlutenPlan { Seq(plan.executeColumnar()) } } + + // When true, it will not generate relNode, nor will it generate native metrics. + def isNoop: Boolean = false } trait LeafTransformSupport extends TransformSupport with LeafExecNode { @@ -130,8 +211,10 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f BackendsApiManager.getMetricsApiInstance.genWholeStageTransformerMetrics(sparkContext) val sparkConf: SparkConf = sparkContext.getConf + val serializableHadoopConf: SerializableConfiguration = new SerializableConfiguration( sparkContext.hadoopConfiguration) + val numaBindingInfo: GlutenNumaBindingInfo = GlutenConfig.getConf.numaBindingInfo @transient @@ -292,10 +375,28 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f */ val allScanPartitions = basicScanExecTransformers.map(_.getPartitions) val allScanSplitInfos = - getSplitInfosFromPartitions( - basicScanExecTransformers, - allScanPartitions, - serializableHadoopConf) + getSplitInfosFromPartitions(basicScanExecTransformers, allScanPartitions) + if (GlutenConfig.getConf.enableHdfsViewfs) { + allScanSplitInfos.foreach { + splitInfos => + splitInfos.foreach { + case splitInfo: LocalFilesNode => + val paths = splitInfo.getPaths.asScala + if (paths.nonEmpty && paths.head.startsWith("viewfs")) { + // Convert the viewfs path into hdfs + val newPaths = paths.map { + viewfsPath => + val viewPath = new Path(viewfsPath) + val viewFileSystem = + FileSystem.get(viewPath.toUri, serializableHadoopConf.value) + viewFileSystem.resolvePath(viewPath).toString + } + splitInfo.setPaths(newPaths.asJava) + } + } + } + } + val inputPartitions = BackendsApiManager.getIteratorApiInstance.genPartitions( wsCtx, @@ -387,8 +488,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f private def getSplitInfosFromPartitions( basicScanExecTransformers: Seq[BasicScanExecTransformer], - allScanPartitions: Seq[Seq[InputPartition]], - serializableHadoopConf: SerializableConfiguration): Seq[Seq[SplitInfo]] = { + allScanPartitions: Seq[Seq[InputPartition]]): Seq[Seq[SplitInfo]] = { // If these are two scan transformers, they must have same partitions, // otherwise, exchange will be inserted. We should combine the two scan // transformers' partitions with same index, and set them together in @@ -407,7 +507,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f val allScanSplitInfos = allScanPartitions.zip(basicScanExecTransformers).map { case (partition, transformer) => - transformer.getSplitInfosFromPartitions(partition, serializableHadoopConf) + transformer.getSplitInfosFromPartitions(partition) } val partitionLength = allScanSplitInfos.head.size if (allScanSplitInfos.exists(_.size != partitionLength)) { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala index 4902b6c6cf1b..28d780992492 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowExecTransformer.scala @@ -51,8 +51,13 @@ case class WindowExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetrics(sparkContext) - override def metricsUpdater(): MetricsUpdater = + override def isNoop: Boolean = windowExpression == null || windowExpression.isEmpty + + override def metricsUpdater(): MetricsUpdater = if (isNoop) { + MetricsUpdater.None + } else { BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) + } override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) @@ -177,17 +182,16 @@ case class WindowExecTransformer( override protected def doTransform(context: SubstraitContext): TransformContext = { val childCtx = child.asInstanceOf[TransformSupport].transform(context) - val operatorId = context.nextOperatorId(this.nodeName) - if (windowExpression == null || windowExpression.isEmpty) { + if (isNoop) { // The computing for this operator is not needed. - context.registerEmptyRelToOperator(operatorId) return childCtx } + val operatorId = context.nextOperatorId(this.nodeName) val currRel = getWindowRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): WindowExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala index 6068412fbad3..5d8a18b11164 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WindowGroupLimitExecTransformer.scala @@ -163,6 +163,6 @@ case class WindowGroupLimitExecTransformer( val currRel = getWindowGroupLimitRel(context, child.output, operatorId, childCtx.root, validation = false) assert(currRel != null, "Window Group Limit Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala index 7034a5a8f0e9..c69623b06193 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/execution/WriteFilesExecTransformer.scala @@ -20,6 +20,7 @@ import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.expression.ConverterUtils import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.enumerated.planner.plan.GlutenPlanModel.GroupLeafExec import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.`type`.ColumnTypeNode import org.apache.gluten.substrait.SubstraitContext @@ -29,12 +30,15 @@ import org.apache.gluten.utils.SubstraitUtil import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, MapType} import org.apache.spark.sql.types.MetadataBuilder import io.substrait.proto.NamedStruct @@ -127,6 +131,37 @@ case class WriteFilesExecTransformer( override protected def doValidateInternal(): ValidationResult = { val finalChildOutput = getFinalChildOutput + + def isConstantComplexType(e: Expression): Boolean = { + e match { + case Literal(_, _: ArrayType | _: MapType) => true + case _ => e.children.exists(isConstantComplexType) + } + } + + lazy val hasConstantComplexType = child match { + case t: ProjectExecTransformer => + t.projectList.exists(isConstantComplexType) + case p: ProjectExec => + p.projectList.exists(isConstantComplexType) + case g: GroupLeafExec => // support the ras + g.metadata + .logicalLink() + .plan + .collectFirst { + case p: Project if p.projectList.exists(isConstantComplexType) => true + } + .isDefined + case _ => false + } + // TODO: currently the velox don't support parquet write with complex data type + // with constant. + if (fileFormat.isInstanceOf[ParquetFileFormat] && hasConstantComplexType) { + return ValidationResult.failed( + "Unsupported native parquet write: " + + "complex data type with constant") + } + val validationResult = BackendsApiManager.getSettings.supportWriteFilesExec( fileFormat, @@ -150,7 +185,7 @@ case class WriteFilesExecTransformer( val currRel = getRelNode(context, getFinalChildOutput, operatorId, childCtx.root, validation = false) assert(currRel != null, "Write Rel should be valid") - TransformContext(childCtx.outputAttributes, output, currRel) + TransformContext(output, currRel) } override protected def withNewChildInternal(newChild: SparkPlan): WriteFilesExecTransformer = diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala index 856d208eada2..b49917c59fcb 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/GlutenPlan.scala @@ -16,21 +16,12 @@ */ package org.apache.gluten.extension -import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.expression.TransformerState -import org.apache.gluten.extension.columnar.transition.Convention -import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.substrait.SubstraitContext -import org.apache.gluten.substrait.plan.PlanBuilder -import org.apache.gluten.substrait.rel.RelNode -import org.apache.gluten.test.TestStats +import org.apache.gluten.extension.columnar.FallbackTag +import org.apache.gluten.extension.columnar.FallbackTag.{Appendable, Converter} +import org.apache.gluten.extension.columnar.FallbackTags.add +import org.apache.gluten.extension.columnar.validator.Validator -import org.apache.spark.sql.execution.SparkPlan - -import com.google.common.collect.Lists +import org.apache.spark.sql.catalyst.trees.TreeNode sealed trait ValidationResult { def ok(): Boolean @@ -38,6 +29,15 @@ sealed trait ValidationResult { } object ValidationResult { + implicit object FromValidationResult extends Converter[ValidationResult] { + override def from(result: ValidationResult): Option[FallbackTag] = { + if (result.ok()) { + return None + } + Some(Appendable(result.reason())) + } + } + private case object Succeeded extends ValidationResult { override def ok(): Boolean = true override def reason(): String = throw new UnsupportedOperationException( @@ -50,82 +50,20 @@ object ValidationResult { def succeeded: ValidationResult = Succeeded def failed(reason: String): ValidationResult = Failed(reason) -} -/** Every Gluten Operator should extend this trait. */ -trait GlutenPlan extends SparkPlan with Convention.KnownBatchType with LogLevelUtil { - protected lazy val enableNativeValidation = glutenConf.enableNativeValidation - - protected def glutenConf: GlutenConfig = GlutenConfig.getConf - - /** - * Validate whether this SparkPlan supports to be transformed into substrait node in Native Code. - */ - final def doValidate(): ValidationResult = { - val schemaVaidationResult = BackendsApiManager.getValidatorApiInstance - .doSchemaValidate(schema) - .map { - reason => - ValidationResult.failed(s"Found schema check failure for $schema, due to: $reason") + implicit class EncodeFallbackTagImplicits(result: ValidationResult) { + def tagOnFallback(plan: TreeNode[_]): Unit = { + if (result.ok()) { + return } - .getOrElse(ValidationResult.succeeded) - if (!schemaVaidationResult.ok()) { - TestStats.addFallBackClassName(this.getClass.toString) - return schemaVaidationResult + add(plan, result) } - try { - TransformerState.enterValidation - val res = doValidateInternal() - if (!res.ok()) { - TestStats.addFallBackClassName(this.getClass.toString) - } - res - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug(s"Just a warning. This exception perhaps needs to be fixed.", e) - } - // FIXME: Use a validation-specific method to catch validation failures - TestStats.addFallBackClassName(this.getClass.toString) - logValidationMessage( - s"Validation failed with exception for plan: $nodeName, due to: ${e.getMessage}", - e) - ValidationResult.failed(e.getMessage) - } finally { - TransformerState.finishValidation - } - } - final override def batchType(): Convention.BatchType = { - if (!supportsColumnar) { - return Convention.BatchType.None - } - val batchType = batchType0() - assert(batchType != Convention.BatchType.None) - batchType - } - - protected def batchType0(): Convention.BatchType = { - Backend.get().defaultBatchType - } - - protected def doValidateInternal(): ValidationResult = ValidationResult.succeeded - - protected def doNativeValidation(context: SubstraitContext, node: RelNode): ValidationResult = { - if (node != null && glutenConf.enableNativeValidation) { - val planNode = PlanBuilder.makePlan(context, Lists.newArrayList(node)) - BackendsApiManager.getValidatorApiInstance - .doNativeValidateWithFailureReason(planNode) - } else { - ValidationResult.succeeded - } - } - - private def logValidationMessage(msg: => String, e: Throwable): Unit = { - if (glutenConf.printStackOnValidationFailure) { - logOnLevel(glutenConf.validationLogLevel, msg, e) - } else { - logOnLevel(glutenConf.validationLogLevel, msg) + def toValidatorOutcome(): Validator.OutCome = { + if (result.ok()) { + return Validator.Passed + } + Validator.Failed(result.reason()) } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala index ff989d796f8e..056315186df1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/EnsureLocalSortRequirements.scala @@ -16,9 +16,7 @@ */ package org.apache.gluten.extension.columnar -import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.rules.Rule @@ -33,24 +31,14 @@ import org.apache.spark.sql.execution.{SortExec, SparkPlan} * SortAggregate with the same key. So, this rule adds local sort back if necessary. */ object EnsureLocalSortRequirements extends Rule[SparkPlan] { - private lazy val offload = TransformPreOverrides.apply() + private lazy val transform: HeuristicTransform = HeuristicTransform.static() private def addLocalSort( originalChild: SparkPlan, requiredOrdering: Seq[SortOrder]): SparkPlan = { + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. val newChild = SortExec(requiredOrdering, global = false, child = originalChild) - if (!GlutenConfig.getConf.enableColumnarSort) { - FallbackTags.add(newChild, "columnar Sort is not enabled in SortExec") - newChild - } else { - val rewrittenPlan = RewriteSparkPlanRulesManager.apply().apply(newChild) - if (rewrittenPlan.eq(newChild) && FallbackTags.nonEmpty(rewrittenPlan)) { - // The sort can not be offloaded - rewrittenPlan - } else { - offload.apply(rewrittenPlan) - } - } + transform.apply(newChild) } override def apply(plan: SparkPlan): SparkPlan = { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala index a5bba46dc605..432ecd1584d8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/FallbackRules.scala @@ -17,147 +17,13 @@ package org.apache.gluten.extension.columnar import org.apache.gluten.GlutenConfig -import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException -import org.apache.gluten.execution._ -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} -import org.apache.gluten.extension.columnar.FallbackTags.EncodeFallbackTagImplicits -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.sql.shims.SparkShimLoader -import org.apache.spark.api.python.EvalPythonExecTransformer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.trees.{TreeNode, TreeNodeTag} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, QueryStageExec} -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange._ import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} -import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -import org.apache.commons.lang3.exception.ExceptionUtils - -sealed trait FallbackTag { - val stacktrace: Option[String] = - if (FallbackTags.DEBUG) { - Some(ExceptionUtils.getStackTrace(new Throwable())) - } else None - - def reason(): String -} - -object FallbackTag { - - /** A tag that stores one reason text of fall back. */ - case class Appendable(override val reason: String) extends FallbackTag - - /** - * A tag that stores reason text of fall back. Other reasons will be discarded when this tag is - * added to plan. - */ - case class Exclusive(override val reason: String) extends FallbackTag - - trait Converter[T] { - def from(obj: T): Option[FallbackTag] - } - - object Converter { - implicit def asIs[T <: FallbackTag]: Converter[T] = (tag: T) => Some(tag) - - implicit object FromString extends Converter[String] { - override def from(reason: String): Option[FallbackTag] = Some(Appendable(reason)) - } - - implicit object FromValidationResult extends Converter[ValidationResult] { - override def from(result: ValidationResult): Option[FallbackTag] = { - if (result.ok()) { - return None - } - Some(Appendable(result.reason())) - } - } - } -} - -object FallbackTags { - val TAG: TreeNodeTag[FallbackTag] = - TreeNodeTag[FallbackTag]("org.apache.gluten.FallbackTag") - - val DEBUG = false - - /** - * If true, the plan node will be guaranteed fallback to Vanilla plan node while being - * implemented. - * - * If false, the plan still has chance to be turned into "non-transformable" in any another - * validation rule. So user should not consider the plan "transformable" unless all validation - * rules are passed. - */ - def nonEmpty(plan: SparkPlan): Boolean = { - getOption(plan).nonEmpty - } - - /** - * If true, it implies the plan maybe transformable during validation phase but not guaranteed, - * since another validation rule could turn it to "non-transformable" before implementing the plan - * within Gluten transformers. If false, the plan node will be guaranteed fallback to Vanilla plan - * node while being implemented. - */ - def maybeOffloadable(plan: SparkPlan): Boolean = !nonEmpty(plan) - - def add[T](plan: TreeNode[_], t: T)(implicit converter: FallbackTag.Converter[T]): Unit = { - val tagOption = getOption(plan) - val newTagOption = converter.from(t) - - val mergedTagOption: Option[FallbackTag] = - (tagOption ++ newTagOption).reduceOption[FallbackTag] { - // New tag comes while the plan was already tagged, merge. - case (_, exclusive: FallbackTag.Exclusive) => - exclusive - case (exclusive: FallbackTag.Exclusive, _) => - exclusive - case (l: FallbackTag.Appendable, r: FallbackTag.Appendable) => - FallbackTag.Appendable(s"${l.reason}; ${r.reason}") - } - mergedTagOption - .foreach(mergedTag => plan.setTagValue(TAG, mergedTag)) - } - - def addRecursively[T](plan: TreeNode[_], t: T)(implicit - converter: FallbackTag.Converter[T]): Unit = { - plan.foreach { - case _: GlutenPlan => // ignore - case other: TreeNode[_] => add(other, t) - } - } - - def untag(plan: TreeNode[_]): Unit = { - plan.unsetTagValue(TAG) - } - - def get(plan: TreeNode[_]): FallbackTag = { - getOption(plan).getOrElse( - throw new IllegalStateException("Transform hint tag not set in plan: " + plan.toString())) - } - - def getOption(plan: TreeNode[_]): Option[FallbackTag] = { - plan.getTagValue(TAG) - } - - implicit class EncodeFallbackTagImplicits(result: ValidationResult) { - def tagOnFallback(plan: TreeNode[_]): Unit = { - if (result.ok()) { - return - } - add(plan, result) - } - } -} case class FallbackOnANSIMode(session: SparkSession) extends Rule[SparkPlan] { override def apply(plan: SparkPlan): SparkPlan = { @@ -169,9 +35,9 @@ case class FallbackOnANSIMode(session: SparkSession) extends Rule[SparkPlan] { } case class FallbackMultiCodegens(session: SparkSession) extends Rule[SparkPlan] { - lazy val columnarConf: GlutenConfig = GlutenConfig.getConf - lazy val physicalJoinOptimize = columnarConf.enablePhysicalJoinOptimize - lazy val optimizeLevel: Integer = columnarConf.physicalJoinOptimizationThrottle + lazy val glutenConf: GlutenConfig = GlutenConfig.getConf + lazy val physicalJoinOptimize = glutenConf.enablePhysicalJoinOptimize + lazy val optimizeLevel: Integer = glutenConf.physicalJoinOptimizationThrottle def existsMultiCodegens(plan: SparkPlan, count: Int = 0): Boolean = plan match { @@ -237,277 +103,3 @@ case class FallbackMultiCodegens(session: SparkSession) extends Rule[SparkPlan] } else plan } } - -// This rule will try to convert a plan into plan transformer. -// The doValidate function will be called to check if the conversion is supported. -// If false is returned or any unsupported exception is thrown, a row guard will -// be added on the top of that plan to prevent actual conversion. -case class AddFallbackTagRule() extends Rule[SparkPlan] { - import AddFallbackTagRule._ - private val glutenConf: GlutenConfig = GlutenConfig.getConf - private val validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnlyWithFilterPushed(glutenConf.enableScanOnly) - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - def apply(plan: SparkPlan): SparkPlan = { - plan.foreachUp { case p => addFallbackTag(p) } - plan - } - - private def addFallbackTag(plan: SparkPlan): Unit = { - val outcome = validator.validate(plan) - outcome match { - case Validator.Failed(reason) => - FallbackTags.add(plan, reason) - return - case Validator.Passed => - } - - try { - plan match { - case plan: BatchScanExec => - // If filter expressions aren't empty, we need to transform the inner operators. - if (plan.runtimeFilters.isEmpty) { - val transformer = - ScanTransformerFactory - .createBatchScanTransformer(plan, validation = true) - .asInstanceOf[BasicScanExecTransformer] - transformer.doValidate().tagOnFallback(plan) - } - case plan: FileSourceScanExec => - // If filter expressions aren't empty, we need to transform the inner operators. - if (plan.partitionFilters.isEmpty) { - val transformer = - ScanTransformerFactory.createFileSourceScanTransformer(plan) - transformer.doValidate().tagOnFallback(plan) - } - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - HiveTableScanExecTransformer.validate(plan).tagOnFallback(plan) - case plan: ProjectExec => - val transformer = ProjectExecTransformer(plan.projectList, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: FilterExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genFilterExecTransformer(plan.condition, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: HashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: SortAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: ObjectHashAggregateExec => - val transformer = HashAggregateExecBaseTransformer.from(plan) - transformer.doValidate().tagOnFallback(plan) - case plan: UnionExec => - val transformer = ColumnarUnionExec(plan.children) - transformer.doValidate().tagOnFallback(plan) - case plan: ExpandExec => - val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: WriteFilesExec => - val transformer = WriteFilesExecTransformer( - plan.child, - plan.fileFormat, - plan.partitionColumns, - plan.bucketSpec, - plan.options, - plan.staticPartitions) - transformer.doValidate().tagOnFallback(plan) - case plan: SortExec => - val transformer = - SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffleExchangeExec => - val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) - transformer.doValidate().tagOnFallback(plan) - case plan: ShuffledHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genShuffledHashJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - OffloadJoin.getShjBuildSide(plan), - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastExchangeExec => - val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) - transformer.doValidate().tagOnFallback(plan) - case bhj: BroadcastHashJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastHashJoinExecTransformer( - bhj.leftKeys, - bhj.rightKeys, - bhj.joinType, - bhj.buildSide, - bhj.condition, - bhj.left, - bhj.right, - isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: SortMergeJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genSortMergeJoinExecTransformer( - plan.leftKeys, - plan.rightKeys, - plan.joinType, - plan.condition, - plan.left, - plan.right, - plan.isSkewJoin) - transformer.doValidate().tagOnFallback(plan) - case plan: CartesianProductExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: BroadcastNestedLoopJoinExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance - .genBroadcastNestedLoopJoinExecTransformer( - plan.left, - plan.right, - plan.buildSide, - plan.joinType, - plan.condition) - transformer.doValidate().tagOnFallback(plan) - case plan: WindowExec => - val transformer = WindowExecTransformer( - plan.windowExpression, - plan.partitionSpec, - plan.orderSpec, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => - val windowGroupLimitPlan = SparkShimLoader.getSparkShims - .getWindowGroupLimitExecShim(plan) - .asInstanceOf[WindowGroupLimitExecShim] - val transformer = WindowGroupLimitExecTransformer( - windowGroupLimitPlan.partitionSpec, - windowGroupLimitPlan.orderSpec, - windowGroupLimitPlan.rankLikeFunction, - windowGroupLimitPlan.limit, - windowGroupLimitPlan.mode, - windowGroupLimitPlan.child - ) - transformer.doValidate().tagOnFallback(plan) - case plan: CoalesceExec => - ColumnarCoalesceExec(plan.numPartitions, plan.child) - .doValidate() - .tagOnFallback(plan) - case plan: GlobalLimitExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - val transformer = LimitTransformer(plan.child, offset, limit) - transformer.doValidate().tagOnFallback(plan) - case plan: LocalLimitExec => - val transformer = LimitTransformer(plan.child, 0L, plan.limit) - transformer.doValidate().tagOnFallback(plan) - case plan: GenerateExec => - val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( - plan.generator, - plan.requiredChildOutput, - plan.outer, - plan.generatorOutput, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: BatchEvalPythonExec => - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - case plan: ArrowEvalPythonExec => - // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not - // enabled, we will try offloading through EvalPythonExecTransformer - if ( - !BackendsApiManager.getSettings.supportColumnarArrowUdf() || - !GlutenConfig.getConf.enableColumnarArrowUDF - ) { - // Both CH and Velox will try using backend's built-in functions for calculate - val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) - transformer.doValidate().tagOnFallback(plan) - } - case plan: TakeOrderedAndProjectExec => - val (limit, offset) = - SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) - val transformer = TakeOrderedAndProjectExecTransformer( - limit, - plan.sortOrder, - plan.projectList, - plan.child, - offset) - transformer.doValidate().tagOnFallback(plan) - case plan: SampleExec => - val transformer = - BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( - plan.lowerBound, - plan.upperBound, - plan.withReplacement, - plan.seed, - plan.child) - transformer.doValidate().tagOnFallback(plan) - case _ => - // Currently we assume a plan to be offload-able by default. - } - } catch { - case e @ (_: GlutenNotSupportException | _: UnsupportedOperationException) => - FallbackTags.add( - plan, - s"${e.getMessage}, original Spark plan is " + - s"${plan.getClass}(${plan.children.toList.map(_.getClass)})") - if (!e.isInstanceOf[GlutenNotSupportException]) { - logDebug("Just a warning. This exception perhaps needs to be fixed.", e) - } - } - } -} - -object AddFallbackTagRule { - implicit private class ValidatorBuilderImplicits(builder: Validators.Builder) { - - /** - * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes - * validation on filter for the exception that filter + scan is detected. Because filters can be - * pushed into scan then the filter conditions will be processed only in scan. - */ - def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validators.Builder = { - builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) - builder - } - } - - private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = { - if (!scanOnly) { - return pass() - } - // Scan-only mode - plan match { - case _: BatchScanExec => pass() - case _: FileSourceScanExec => pass() - case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() - case filter: FilterExec => - val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || - filter.child.isInstanceOf[BatchScanExec] - if (childIsScan) { - pass() - } else { - fail(filter) - } - case other => fail(other) - } - } - } -} - -case class RemoveFallbackTagRule() extends Rule[SparkPlan] { - override def apply(plan: SparkPlan): SparkPlan = { - plan.foreach(FallbackTags.untag) - plan - } -} diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala similarity index 92% rename from backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala index 63c5fe017f5e..a034a3229a88 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/extension/MergeTwoPhasesHashBaseAggregate.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MergeTwoPhasesHashBaseAggregate.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension +package org.apache.gluten.extension.columnar import org.apache.gluten.GlutenConfig @@ -38,10 +38,11 @@ case class MergeTwoPhasesHashBaseAggregate(session: SparkSession) extends Rule[SparkPlan] with Logging { - val columnarConf: GlutenConfig = GlutenConfig.getConf - val scanOnly: Boolean = columnarConf.enableScanOnly - val enableColumnarHashAgg: Boolean = !scanOnly && columnarConf.enableColumnarHashAgg + val glutenConf: GlutenConfig = GlutenConfig.getConf + val scanOnly: Boolean = glutenConf.enableScanOnly + val enableColumnarHashAgg: Boolean = !scanOnly && glutenConf.enableColumnarHashAgg val replaceSortAggWithHashAgg: Boolean = GlutenConfig.getConf.forceToUseHashAgg + val mergeTwoPhasesAggEnabled: Boolean = GlutenConfig.getConf.mergeTwoPhasesAggEnabled private def isPartialAgg(partialAgg: BaseAggregateExec, finalAgg: BaseAggregateExec): Boolean = { // TODO: now it can not support to merge agg which there are the filters in the aggregate exprs. @@ -59,7 +60,7 @@ case class MergeTwoPhasesHashBaseAggregate(session: SparkSession) } override def apply(plan: SparkPlan): SparkPlan = { - if (!enableColumnarHashAgg) { + if (!mergeTwoPhasesAggEnabled || !enableColumnarHashAgg) { plan } else { plan.transformDown { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala index 11b4b8650842..e11c6139547a 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/MiscColumnarRules.scala @@ -16,8 +16,8 @@ */ package org.apache.gluten.extension.columnar +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, Transitions} -import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.SparkSession @@ -32,34 +32,6 @@ import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.internal.SQLConf object MiscColumnarRules { - object TransformPreOverrides { - def apply(): TransformPreOverrides = { - TransformPreOverrides( - List(), - List( - OffloadOthers(), - OffloadExchange(), - OffloadJoin() - ) - ) - } - } - - // This rule will conduct the conversion from Spark plan to the plan transformer. - case class TransformPreOverrides( - topDownRules: Seq[OffloadSingleNode], - bottomUpRules: Seq[OffloadSingleNode]) - extends Rule[SparkPlan] - with LogLevelUtil { - - def apply(plan: SparkPlan): SparkPlan = { - val plan0 = - topDownRules.foldLeft(plan)((p, rule) => p.transformDown { case p => rule.offload(p) }) - val plan1 = - bottomUpRules.foldLeft(plan0)((p, rule) => p.transformUp { case p => rule.offload(p) }) - plan1 - } - } // Replaces all SubqueryBroadcastExec used by sub-queries with ColumnarSubqueryBroadcastExec. // This prevents query execution from being failed by fallen-back SubqueryBroadcastExec with @@ -135,7 +107,8 @@ object MiscColumnarRules { private def toColumnarBroadcastExchange( exchange: BroadcastExchangeExec): ColumnarBroadcastExchangeExec = { - val newChild = Transitions.toBackendBatchPlan(exchange.child) + val newChild = + Transitions.toBatchPlan(exchange.child, BackendsApiManager.getSettings.primaryBatchType) ColumnarBroadcastExchangeExec(exchange.mode, newChild) } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala new file mode 100644 index 000000000000..b8972b6d0e2c --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/PruneNestedColumnsInHiveTableScan.scala @@ -0,0 +1,33 @@ +/* + * 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.gluten.extension.columnar + +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.hive.HiveTableScanNestedColumnPruning + +// Since https://github.com/apache/incubator-gluten/pull/7268. +// Used only by CH backend as of now. +object PruneNestedColumnsInHiveTableScan extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case p: ProjectExecTransformer + if HiveTableScanNestedColumnPruning.supportNestedColumnPruning(p) => + HiveTableScanNestedColumnPruning.apply(p) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala new file mode 100644 index 000000000000..f0eea08018dd --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/UnionTransformerRule.scala @@ -0,0 +1,61 @@ +/* + * 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.gluten.extension.columnar + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution.{ColumnarUnionExec, UnionExecTransformer} + +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * Replace ColumnarUnionExec with UnionExecTransformer if possible. + * + * The rule is not included in [[org.apache.gluten.extension.columnar.heuristic.HeuristicTransform]] + * or [[org.apache.gluten.extension.columnar.enumerated.EnumeratedTransform]] because it relies on + * children's output partitioning to be fully provided. + */ +case class UnionTransformerRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = { + if (!GlutenConfig.getConf.enableNativeUnion) { + return plan + } + plan.transformUp { + case plan: ColumnarUnionExec => + val transformer = UnionExecTransformer(plan.children) + if (sameNumPartitions(plan.children) && validate(transformer)) { + transformer + } else { + plan + } + } + } + + private def sameNumPartitions(plans: Seq[SparkPlan]): Boolean = { + val partitioning = plans.map(_.outputPartitioning) + if (partitioning.exists(p => p.isInstanceOf[UnknownPartitioning])) { + return false + } + val numPartitions = plans.map(_.outputPartitioning.numPartitions) + numPartitions.forall(_ == numPartitions.head) + } + + private def validate(union: UnionExecTransformer): Boolean = { + union.doValidate().ok() + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala deleted file mode 100644 index e6d1c4859ed4..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/EnumeratedTransform.scala +++ /dev/null @@ -1,115 +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. - */ -package org.apache.gluten.extension.columnar.enumerated - -import org.apache.gluten.extension.columnar.{OffloadExchange, OffloadJoin, OffloadOthers} -import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.validator.{Validator, Validators} -import org.apache.gluten.logging.LogLevelUtil -import org.apache.gluten.planner.GlutenOptimization -import org.apache.gluten.planner.cost.GlutenCostModel -import org.apache.gluten.planner.property.Conv -import org.apache.gluten.ras.property.PropertySet -import org.apache.gluten.sql.shims.SparkShimLoader - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.WriteFilesExec -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase -import org.apache.spark.sql.execution.exchange.Exchange -import org.apache.spark.sql.execution.joins.BaseJoinExec -import org.apache.spark.sql.execution.python.EvalPythonExec -import org.apache.spark.sql.execution.window.WindowExec -import org.apache.spark.sql.hive.HiveTableScanExecTransformer - -case class EnumeratedTransform(session: SparkSession, outputsColumnar: Boolean) - extends Rule[SparkPlan] - with LogLevelUtil { - - private val validator: Validator = Validators - .builder() - .fallbackByHint() - .fallbackIfScanOnly() - .fallbackComplexExpressions() - .fallbackByBackendSettings() - .fallbackByUserOptions() - .fallbackByTestInjects() - .build() - - private val rules = List( - RemoveSort - ) - - // TODO: Should obey ReplaceSingleNode#applyScanNotTransformable to select - // (vanilla) scan with cheaper sub-query plan through cost model. - private val offloadRules = - Seq( - RasOffload.from[Exchange](OffloadExchange()), - RasOffload.from[BaseJoinExec](OffloadJoin()), - RasOffload.from[FilterExec](OffloadOthers()), - RasOffload.from[ProjectExec](OffloadOthers()), - RasOffload.from[DataSourceV2ScanExecBase](OffloadOthers()), - RasOffload.from[DataSourceScanExec](OffloadOthers()), - RasOffload - .from(HiveTableScanExecTransformer.isHiveTableScan, OffloadOthers()), - RasOffload.from[CoalesceExec](OffloadOthers()), - RasOffload.from[HashAggregateExec](OffloadOthers()), - RasOffload.from[SortAggregateExec](OffloadOthers()), - RasOffload.from[ObjectHashAggregateExec](OffloadOthers()), - RasOffload.from[UnionExec](OffloadOthers()), - RasOffload.from[ExpandExec](OffloadOthers()), - RasOffload.from[WriteFilesExec](OffloadOthers()), - RasOffload.from[SortExec](OffloadOthers()), - RasOffload.from[TakeOrderedAndProjectExec](OffloadOthers()), - RasOffload.from[WindowExec](OffloadOthers()), - RasOffload - .from(SparkShimLoader.getSparkShims.isWindowGroupLimitExec, OffloadOthers()), - RasOffload.from[LimitExec](OffloadOthers()), - RasOffload.from[GenerateExec](OffloadOthers()), - RasOffload.from[EvalPythonExec](OffloadOthers()), - RasOffload.from[SampleExec](OffloadOthers()) - ).map(RasOffload.Rule(_, validator)) - - private val optimization = { - GlutenOptimization - .builder() - .costModel(GlutenCostModel.find()) - .addRules(rules ++ offloadRules) - .create() - } - - private val reqConvention = Conv.any - - private val altConventions = { - val rowBased: Conv = Conv.req(ConventionReq.row) - val backendBatchBased: Conv = Conv.req(ConventionReq.backendBatch) - Seq(rowBased, backendBatchBased) - } - - override def apply(plan: SparkPlan): SparkPlan = { - val constraintSet = PropertySet(List(reqConvention)) - val altConstraintSets = - altConventions.map(altConv => PropertySet(List(altConv))) - val planner = optimization.newPlanner(plan, constraintSet, altConstraintSets) - val out = planner.plan() - out - } -} - -object EnumeratedTransform {} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala index 52798f712c12..982abc16c527 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RasOffload.scala @@ -16,8 +16,8 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.OffloadSingleNode +import org.apache.gluten.execution.{GlutenPlan, ValidatablePlan} +import org.apache.gluten.extension.columnar.offload.OffloadSingleNode import org.apache.gluten.extension.columnar.rewrite.RewriteSingleNode import org.apache.gluten.extension.columnar.validator.Validator import org.apache.gluten.ras.path.Pattern @@ -50,8 +50,6 @@ object RasOffload { } } - private val rewrites = RewriteSingleNode.allRules() - def from[T <: SparkPlan: ClassTag](base: OffloadSingleNode): RasOffload = { new RasOffload { override def offload(plan: SparkPlan): SparkPlan = base.offload(plan) @@ -59,19 +57,24 @@ object RasOffload { } } - def from(identifier: TypeIdentifier, base: OffloadSingleNode): RasOffload = { + def from(identifier: SparkPlan => Boolean)(base: OffloadSingleNode): RasOffload = { new RasOffload { override def offload(plan: SparkPlan): SparkPlan = base.offload(plan) - override def typeIdentifier(): TypeIdentifier = identifier + override def typeIdentifier(): TypeIdentifier = new TypeIdentifier { + override def isInstance(node: SparkPlan): Boolean = identifier(node) + } } } object Rule { - def apply(base: RasOffload, validator: Validator): RasRule[SparkPlan] = { - new RuleImpl(base, validator) + def apply( + base: RasOffload, + validator: Validator, + rewrites: Seq[RewriteSingleNode]): RasRule[SparkPlan] = { + new RuleImpl(base, validator, rewrites) } - private class RuleImpl(base: RasOffload, validator: Validator) + private class RuleImpl(base: RasOffload, validator: Validator, rewrites: Seq[RewriteSingleNode]) extends RasRule[SparkPlan] with Logging { private val typeIdentifier: TypeIdentifier = base.typeIdentifier() @@ -116,10 +119,12 @@ object RasOffload { validator.validate(from) match { case Validator.Passed => val offloadedPlan = base.offload(from) - val offloadedNodes = offloadedPlan.collect[GlutenPlan] { case t: GlutenPlan => t } + val offloadedNodes = offloadedPlan.collect[ValidatablePlan] { + case t: ValidatablePlan => t + } val outComes = offloadedNodes.map(_.doValidate()).filter(!_.ok()) if (outComes.nonEmpty) { - // 4. If native validation fails on at least one of the offloaded nodes, return + // 5. If native validation fails on at least one of the offloaded nodes, return // the original one. // // TODO: Tag the original plan with fallback reason. This is a non-trivial work @@ -130,7 +135,9 @@ object RasOffload { offloadedPlan } case Validator.Failed(reason) => - // TODO: Tag the original plan with fallback reason. + // TODO: Tag the original plan with fallback reason. This is a non-trivial work + // in RAS as the query plan we got here may be a copy so may not propagate tags + // to original plan. from } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala index 5b5d5e541eb7..fd562406162e 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/RemoveSort.scala @@ -16,8 +16,7 @@ */ package org.apache.gluten.extension.columnar.enumerated -import org.apache.gluten.execution.{HashAggregateExecBaseTransformer, ShuffledHashJoinExecTransformerBase, SortExecTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, HashAggregateExecBaseTransformer, ShuffledHashJoinExecTransformerBase, SortExecTransformer} import org.apache.gluten.ras.path.Pattern._ import org.apache.gluten.ras.path.Pattern.Matchers._ import org.apache.gluten.ras.rule.{RasRule, Shape} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala similarity index 85% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala index 2e32e3f06a4b..5cf9b87f2ac1 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LegacyCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/LegacyCoster.scala @@ -14,23 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.extension.columnar.transition.{ColumnarToColumnarLike, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.execution.{ColumnarToRowExec, ColumnarWriteFilesExec, ProjectExec, RowToColumnarExec, SparkPlan} -/** - * A cost model that is supposed to drive RAS planner create the same query plan with legacy - * planner. - */ -class LegacyCostModel extends LongCostModel { +object LegacyCoster extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Legacy + + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) + } // A very rough estimation as of now. The cost model basically considers any // fallen back ops as having extreme high cost so offloads computations as // much as possible. - override def selfLongCostOf(node: SparkPlan): Long = { + private def selfCostOf0(node: SparkPlan): Long = { node match { case ColumnarWriteFilesExec.OnNoopLeafPath(_) => 0 case ColumnarToRowExec(_) => 10L diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala similarity index 87% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala index 2576a1008472..d2959d46a13c 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster.scala @@ -14,20 +14,24 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.execution.RowToColumnarExecBase import org.apache.gluten.extension.columnar.transition.{ColumnarToColumnarLike, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression} -import org.apache.spark.sql.execution.{ColumnarToRowExec, ColumnarWriteFilesExec, ProjectExec, RowToColumnarExec, SparkPlan} +import org.apache.spark.sql.execution._ import org.apache.spark.sql.types.{ArrayType, MapType, StructType} -/** A rough cost model with some empirical heuristics. */ -class RoughCostModel extends LongCostModel { +object RoughCoster extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Rough - override def selfLongCostOf(node: SparkPlan): Long = { + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) + } + + private def selfCostOf0(node: SparkPlan): Long = { node match { case ColumnarWriteFilesExec.OnNoopLeafPath(_) => 0 case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala similarity index 88% rename from gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala index 4452e777980c..832c524d7b55 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/RoughCostModel2.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/enumerated/planner/cost/RoughCoster2.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.planner.cost +package org.apache.gluten.extension.columnar.enumerated.planner.cost import org.apache.gluten.GlutenConfig import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike} @@ -25,31 +25,15 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExecBase -/** Compared with rough, rough2 can be more precise to avoid the costly r2c. */ -class RoughCostModel2 extends LongCostModel { +// Since https://github.com/apache/incubator-gluten/pull/7686. +object RoughCoster2 extends LongCoster { + override def kind(): LongCostModel.Kind = LongCostModel.Rough2 - private def getSizeFactor(plan: SparkPlan): Long = { - // Get the bytes size that the plan needs to consume. - val sizeBytes = plan match { - case _: DataSourceScanExec | _: DataSourceV2ScanExecBase => getStatSizeBytes(plan) - case _: LeafExecNode => 0L - case p => p.children.map(getStatSizeBytes).sum - } - sizeBytes / GlutenConfig.getConf.rasRough2SizeBytesThreshold - } - - private def getStatSizeBytes(plan: SparkPlan): Long = { - plan match { - case a: AdaptiveSparkPlanExec => getStatSizeBytes(a.inputPlan) - case _ => - plan.logicalLink match { - case Some(logicalPlan) => logicalPlan.stats.sizeInBytes.toLong - case _ => plan.children.map(getStatSizeBytes).sum - } - } + override def selfCostOf(node: SparkPlan): Option[Long] = { + Some(selfCostOf0(node)) } - override def selfLongCostOf(node: SparkPlan): Long = { + private def selfCostOf0(node: SparkPlan): Long = { val sizeFactor = getSizeFactor(node) val opCost = node match { case ProjectExec(projectList, _) if projectList.forall(isCheapExpression) => @@ -70,6 +54,27 @@ class RoughCostModel2 extends LongCostModel { opCost * Math.max(1, sizeFactor) } + private def getSizeFactor(plan: SparkPlan): Long = { + // Get the bytes size that the plan needs to consume. + val sizeBytes = plan match { + case _: DataSourceScanExec | _: DataSourceV2ScanExecBase => getStatSizeBytes(plan) + case _: LeafExecNode => 0L + case p => p.children.map(getStatSizeBytes).sum + } + sizeBytes / GlutenConfig.getConf.rasRough2SizeBytesThreshold + } + + private def getStatSizeBytes(plan: SparkPlan): Long = { + plan match { + case a: AdaptiveSparkPlanExec => getStatSizeBytes(a.inputPlan) + case _ => + plan.logicalLink match { + case Some(logicalPlan) => logicalPlan.stats.sizeInBytes.toLong + case _ => plan.children.map(getStatSizeBytes).sum + } + } + } + private def isCheapExpression(ne: NamedExpression): Boolean = ne match { case Alias(_: Attribute, _) => true case _: Attribute => true diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala similarity index 92% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala index 9c0ddac16b76..3418d3dddc99 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/ExpandFallbackPolicy.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/heuristic/ExpandFallbackPolicy.scala @@ -14,22 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar +package org.apache.gluten.extension.columnar.heuristic import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.heuristic.FallbackNode -import org.apache.gluten.extension.columnar.transition.{ColumnarToRowLike, RowToColumnarLike, Transitions} +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.{FallbackTag, FallbackTags} +import org.apache.gluten.extension.columnar.FallbackTags.add +import org.apache.gluten.extension.columnar.transition.{BackendTransitions, ColumnarToRowLike, RowToColumnarLike} import org.apache.gluten.utils.PlanUtil - import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec import org.apache.spark.sql.execution.exchange.Exchange -// spotless:off + + + + + +// format: off /** * Note, this rule should only fallback to row-based plan if there is no harm. * The follow case should be handled carefully @@ -62,9 +68,10 @@ import org.apache.spark.sql.execution.exchange.Exchange * @param isAdaptiveContext If is inside AQE * @param originalPlan The vanilla SparkPlan without apply gluten transform rules */ -// spotless:on +// format: on case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkPlan) extends Rule[SparkPlan] { + import ExpandFallbackPolicy._ private def countTransitionCost(plan: SparkPlan): Int = { val ignoreRowToColumnar = GlutenConfig.getConf.fallbackIgnoreRowToColumnar @@ -103,14 +110,13 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP transitionCost } + // format: off /** * When making a stage fall back, it's possible that we need a ColumnarToRow to adapt to last * stage's columnar output. So we need to evaluate the cost, i.e., the number of required * ColumnarToRow between entirely fallback stage and last stage(s). Thus, we can avoid possible * performance degradation caused by fallback policy. * - * spotless:off - * * Spark plan before applying fallback policy: * * ColumnarExchange @@ -133,9 +139,8 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP * Project * * So by considering the cost, the fallback policy will not be applied. - * - * spotless:on */ + // format: on private def countStageFallbackTransitionCost(plan: SparkPlan): Int = { var stageFallbackTransitionCost = 0 @@ -224,7 +229,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP case _ => } - val planWithTransitions = Transitions.insertTransitions(originalPlan, outputsColumnar) + val planWithTransitions = BackendTransitions.insert(originalPlan, outputsColumnar) planWithTransitions } @@ -255,7 +260,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP ) { plan } else { - FallbackTags.addRecursively( + addFallbackTagsRecursively( vanillaSparkPlan, FallbackTag.Exclusive(fallbackInfo.reason.getOrElse("Unknown reason"))) FallbackNode(vanillaSparkPlan) @@ -273,3 +278,13 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP def DO_NOT_FALLBACK(): FallbackInfo = FallbackInfo() } } + +object ExpandFallbackPolicy { + private def addFallbackTagsRecursively[T](plan: TreeNode[_], t: T)(implicit + converter: FallbackTag.Converter[T]): Unit = { + plan.foreach { + case _: GlutenPlan => // ignore + case other: TreeNode[_] => add(other, t) + } + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala similarity index 84% rename from gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala rename to gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala index ce5b21206986..7dc40faa4315 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/OffloadSingleNode.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/offload/OffloadSingleNodeRules.scala @@ -14,13 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.gluten.extension.columnar +package org.apache.gluten.extension.columnar.offload import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.exception.GlutenNotSupportException import org.apache.gluten.execution._ -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil import org.apache.gluten.sql.shims.SparkShimLoader @@ -38,17 +37,6 @@ import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPyth import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} import org.apache.spark.sql.hive.HiveTableScanExecTransformer -/** - * Converts a vanilla Spark plan node into Gluten plan node. Gluten plan is supposed to be executed - * in native, and the internals of execution is subject by backend's implementation. - * - * Note: Only the current plan node is supposed to be open to modification. Do not access or modify - * the children node. Tree-walking is done by caller of this trait. - */ -sealed trait OffloadSingleNode extends Logging { - def offload(plan: SparkPlan): SparkPlan -} - // Exchange transformation. case class OffloadExchange() extends OffloadSingleNode with LogLevelUtil { override def offload(plan: SparkPlan): SparkPlan = plan match { @@ -189,7 +177,7 @@ object OffloadJoin { // Other transformations. case class OffloadOthers() extends OffloadSingleNode with LogLevelUtil { import OffloadOthers._ - private val replace = new ReplaceSingleNode() + private val replace = new ReplaceSingleNode override def offload(plan: SparkPlan): SparkPlan = replace.doReplace(plan) } @@ -198,10 +186,10 @@ object OffloadOthers { // Utility to replace single node within transformed Gluten node. // Children will be preserved as they are as children of the output node. // - // Do not look-up on children on the input node in this rule. Otherwise + // Do not look up on children on the input node in this rule. Otherwise // it may break RAS which would group all the possible input nodes to // search for validate candidates. - private class ReplaceSingleNode() extends LogLevelUtil with Logging { + private class ReplaceSingleNode extends LogLevelUtil with Logging { def doReplace(p: SparkPlan): SparkPlan = { val plan = p @@ -210,11 +198,15 @@ object OffloadOthers { } plan match { case plan: BatchScanExec => - applyScanTransformer(plan) + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + ScanTransformerFactory.createBatchScanTransformer(plan) case plan: FileSourceScanExec => - applyScanTransformer(plan) + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + ScanTransformerFactory.createFileSourceScanTransformer(plan) case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - applyScanTransformer(plan) + // TODO: Add DynamicPartitionPruningHiveScanSuite.scala + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") + HiveTableScanExecTransformer(plan) case plan: CoalesceExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") ColumnarCoalesceExec(plan.numPartitions, plan.child) @@ -253,7 +245,6 @@ object OffloadOthers { plan.bucketSpec, plan.options, plan.staticPartitions) - ColumnarWriteFilesExec( writeTransformer, plan.fileFormat, @@ -298,11 +289,11 @@ object OffloadOthers { val child = plan.child val (limit, offset) = SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) - LimitTransformer(child, offset, limit) + LimitExecTransformer(child, offset, limit) case plan: LocalLimitExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") val child = plan.child - LimitTransformer(child, 0L, plan.limit) + LimitExecTransformer(child, 0L, plan.limit) case plan: GenerateExec => logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") val child = plan.child @@ -341,45 +332,9 @@ object OffloadOthers { child) case p if !p.isInstanceOf[GlutenPlan] => logDebug(s"Transformation for ${p.getClass} is currently not supported.") - val children = plan.children - p.withNewChildren(children) + p case other => other } } - - /** - * Apply scan transformer for file source and batch source, - * 1. create new filter and scan transformer, 2. validate, tag new scan as unsupported if - * failed, 3. return new source. - */ - private def applyScanTransformer(plan: SparkPlan): SparkPlan = plan match { - case plan: FileSourceScanExec => - val transformer = ScanTransformerFactory.createFileSourceScanTransformer(plan) - val validationResult = transformer.doValidate() - if (validationResult.ok()) { - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - transformer - } else { - logDebug(s"Columnar Processing for ${plan.getClass} is currently unsupported.") - FallbackTags.add(plan, validationResult.reason()) - plan - } - case plan: BatchScanExec => - ScanTransformerFactory.createBatchScanTransformer(plan) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - // TODO: Add DynamicPartitionPruningHiveScanSuite.scala - val hiveTableScanExecTransformer = - BackendsApiManager.getSparkPlanExecApiInstance.genHiveTableScanExecTransformer(plan) - val validateResult = hiveTableScanExecTransformer.doValidate() - if (validateResult.ok()) { - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - return hiveTableScanExecTransformer - } - logDebug(s"Columnar Processing for ${plan.getClass} is currently unsupported.") - FallbackTags.add(plan, validateResult.reason()) - plan - case other => - throw new GlutenNotSupportException(s"${other.getClass.toString} is not supported.") - } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala index 6ede36446ef1..1a3ecca16a00 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPostProject.scala @@ -33,6 +33,9 @@ import scala.collection.mutable.ArrayBuffer * when a fallback occurs. */ object PullOutPostProject extends RewriteSingleNode with PullOutProjectHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def needsPostProjection(plan: SparkPlan): Boolean = { plan match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala index 51cdb76a1559..7f32014c2478 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/PullOutPreProject.scala @@ -37,6 +37,9 @@ import scala.collection.mutable * execution by the native engine. */ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def needsPreProject(plan: SparkPlan): Boolean = { plan match { @@ -222,7 +225,13 @@ object PullOutPreProject extends RewriteSingleNode with PullOutProjectHelper { case expand: ExpandExec if needsPreProject(expand) => val expressionMap = new mutable.HashMap[Expression, NamedExpression]() val newProjections = - expand.projections.map(_.map(replaceExpressionWithAttribute(_, expressionMap))) + expand.projections.map( + _.map( + replaceExpressionWithAttribute( + _, + expressionMap, + replaceBoundReference = false, + replaceLiteral = false))) expand.copy( projections = newProjections, child = ProjectExec( diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala new file mode 100644 index 000000000000..dd2847df6ff8 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteEligibility.scala @@ -0,0 +1,48 @@ +/* + * 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.gluten.extension.columnar.rewrite + +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.sql.execution.{ExpandExec, FileSourceScanExec, FilterExec, GenerateExec, SortExec, SparkPlan, TakeOrderedAndProjectExec} +import org.apache.spark.sql.execution.aggregate.BaseAggregateExec +import org.apache.spark.sql.execution.joins.BaseJoinExec +import org.apache.spark.sql.execution.python.ArrowEvalPythonExec +import org.apache.spark.sql.execution.window.WindowExec + +/** + * TODO: Remove this then implement API #isRewritable in rewrite rules. + * + * Since https://github.com/apache/incubator-gluten/pull/4645 + */ +object RewriteEligibility { + def isRewritable(plan: SparkPlan): Boolean = plan match { + case _: SortExec => true + case _: TakeOrderedAndProjectExec => true + case _: BaseAggregateExec => true + case _: BaseJoinExec => true + case _: WindowExec => true + case _: FilterExec => true + case _: FileSourceScanExec => true + case _: ExpandExec => true + case _: GenerateExec => true + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => true + case _: ArrowEvalPythonExec => true + case _ => false + } + +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala index da120c39a4a8..5a28576750ab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteIn.scala @@ -32,6 +32,9 @@ import org.apache.spark.sql.types.StructType * TODO: Remove this rule once Velox support the list option in `In` is not literal. */ object RewriteIn extends RewriteSingleNode { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } private def shouldRewrite(e: Expression): Boolean = { e match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala index d0cac0b29deb..2d844a616c5b 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteJoin.scala @@ -17,7 +17,7 @@ package org.apache.gluten.extension.columnar.rewrite import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.columnar.OffloadJoin +import org.apache.gluten.extension.columnar.offload.OffloadJoin import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, JoinSelectionHelper} import org.apache.spark.sql.catalyst.plans.logical.Join @@ -26,6 +26,10 @@ import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoin /** If force ShuffledHashJoin, convert [[SortMergeJoinExec]] to [[ShuffledHashJoinExec]]. */ object RewriteJoin extends RewriteSingleNode with JoinSelectionHelper { + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } + private def getSmjBuildSide(join: SortMergeJoinExec): Option[BuildSide] = { val leftBuildable = canBuildShuffledHashJoinLeft(join.joinType) val rightBuildable = canBuildShuffledHashJoinRight(join.joinType) diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala index b395d961a075..1d11dcc91795 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/rewrite/RewriteMultiChildrenCount.scala @@ -48,6 +48,10 @@ import org.apache.spark.sql.types.IntegerType object RewriteMultiChildrenCount extends RewriteSingleNode with PullOutProjectHelper { private lazy val shouldRewriteCount = BackendsApiManager.getSettings.shouldRewriteCount() + override def isRewritable(plan: SparkPlan): Boolean = { + RewriteEligibility.isRewritable(plan) + } + private def extractCountForRewrite(aggExpr: AggregateExpression): Option[Count] = { val isPartialCountWithMoreThanOneChild = aggExpr.mode == Partial && { aggExpr.aggregateFunction match { diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala new file mode 100644 index 000000000000..86d4b40d55d8 --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/transition/BackendTransitions.scala @@ -0,0 +1,36 @@ +/* + * 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.gluten.extension.columnar.transition + +import org.apache.gluten.backendsapi.BackendsApiManager + +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +case class InsertBackendTransitions(outputsColumnar: Boolean) extends Rule[SparkPlan] { + def apply(plan: SparkPlan): SparkPlan = { + InsertTransitions + .create(outputsColumnar, BackendsApiManager.getSettings.primaryBatchType) + .apply(plan) + } +} + +object BackendTransitions { + def insert(plan: SparkPlan, outputsColumnar: Boolean): SparkPlan = { + InsertBackendTransitions(outputsColumnar)(plan) + } +} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala index 50201efc07d2..d246167bd7c8 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/extension/columnar/validator/Validators.scala @@ -18,34 +18,32 @@ package org.apache.gluten.extension.columnar.validator import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.{BackendsApiManager, BackendSettingsApi} +import org.apache.gluten.execution._ import org.apache.gluten.expression.ExpressionUtils import org.apache.gluten.extension.columnar.FallbackTags +import org.apache.gluten.extension.columnar.offload.OffloadJoin import org.apache.gluten.sql.shims.SparkShimLoader +import org.apache.spark.api.python.EvalPythonExecTransformer +import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.execution.window.WindowExec +import org.apache.spark.sql.execution.python.{ArrowEvalPythonExec, BatchEvalPythonExec} +import org.apache.spark.sql.execution.window.{WindowExec, WindowGroupLimitExecShim} import org.apache.spark.sql.hive.HiveTableScanExecTransformer -import scala.collection.mutable -import scala.collection.mutable.ListBuffer - object Validators { - def builder(): Builder = Builder() - - class Builder private { + implicit class ValidatorBuilderImplicits(builder: Validator.Builder) { private val conf = GlutenConfig.getConf private val settings = BackendsApiManager.getSettings - private val buffer: ListBuffer[Validator] = mutable.ListBuffer() /** Fails validation if a plan node was already tagged with TRANSFORM_UNSUPPORTED. */ - def fallbackByHint(): Builder = { - buffer += FallbackByHint - this + def fallbackByHint(): Validator.Builder = { + builder.add(FallbackByHint) } /** @@ -53,59 +51,53 @@ object Validators { * executed by native library. By default, we use a threshold option in config to make the * decision. */ - def fallbackComplexExpressions(): Builder = { - buffer += new FallbackComplexExpressions(conf.fallbackExpressionsThreshold) - this + def fallbackComplexExpressions(): Validator.Builder = { + builder.add(new FallbackComplexExpressions(conf.fallbackExpressionsThreshold)) } /** Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. */ - def fallbackIfScanOnly(): Builder = { - buffer += new FallbackIfScanOnly(conf.enableScanOnly) - this + def fallbackIfScanOnly(): Validator.Builder = { + builder.add(new FallbackIfScanOnly(conf.enableScanOnly)) } /** * Fails validation if native-execution of a plan node is not supported by current backend * implementation by checking the active BackendSettings. */ - def fallbackByBackendSettings(): Builder = { - buffer += new FallbackByBackendSettings(settings) - this + def fallbackByBackendSettings(): Validator.Builder = { + builder.add(new FallbackByBackendSettings(settings)) } /** * Fails validation if native-execution of a plan node is disabled by Gluten/Spark * configuration. */ - def fallbackByUserOptions(): Builder = { - buffer += new FallbackByUserOptions(conf) - this + def fallbackByUserOptions(): Validator.Builder = { + builder.add(new FallbackByUserOptions(conf)) } - def fallbackByTestInjects(): Builder = { - buffer += new FallbackByTestInjects() - this + def fallbackByTestInjects(): Validator.Builder = { + builder.add(new FallbackByTestInjects()) } - /** Add a custom validator to pipeline. */ - def add(validator: Validator): Builder = { - buffer += validator - this + /** + * Fails validation on non-scan plan nodes if Gluten is running as scan-only mode. Also, passes + * validation on filter for the exception that filter + scan is detected. Because filters can be + * pushed into scan then the filter conditions will be processed only in scan. + */ + def fallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean): Validator.Builder = { + builder.add(new FallbackIfScanOnlyWithFilterPushed(scanOnly)) } - def build(): Validator = { - if (buffer.isEmpty) { - NoopValidator - } else { - new ValidatorPipeline(buffer.toSeq) - } + /** + * Attempts to offload the input query plan node and check native validation result. Fails when + * native validation failed. + */ + def fallbackByNativeValidation(): Validator.Builder = { + builder.add(new FallbackByNativeValidation) } } - private object Builder { - def apply(): Builder = new Builder() - } - private object FallbackByHint extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = { if (FallbackTags.nonEmpty(plan)) { @@ -149,47 +141,50 @@ object Validators { } } - private class FallbackByUserOptions(conf: GlutenConfig) extends Validator { + private class FallbackByUserOptions(glutenConf: GlutenConfig) extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = plan match { - case p: SortExec if !conf.enableColumnarSort => fail(p) - case p: WindowExec if !conf.enableColumnarWindow => fail(p) - case p: SortMergeJoinExec if !conf.enableColumnarSortMergeJoin => fail(p) - case p: BatchScanExec if !conf.enableColumnarBatchScan => fail(p) - case p: FileSourceScanExec if !conf.enableColumnarFileScan => fail(p) - case p: ProjectExec if !conf.enableColumnarProject => fail(p) - case p: FilterExec if !conf.enableColumnarFilter => fail(p) - case p: UnionExec if !conf.enableColumnarUnion => fail(p) - case p: ExpandExec if !conf.enableColumnarExpand => fail(p) - case p: SortAggregateExec if !conf.forceToUseHashAgg => fail(p) - case p: ShuffledHashJoinExec if !conf.enableColumnarShuffledHashJoin => fail(p) - case p: ShuffleExchangeExec if !conf.enableColumnarShuffle => fail(p) - case p: BroadcastExchangeExec if !conf.enableColumnarBroadcastExchange => fail(p) - case p @ (_: LocalLimitExec | _: GlobalLimitExec) if !conf.enableColumnarLimit => fail(p) - case p: GenerateExec if !conf.enableColumnarGenerate => fail(p) - case p: CoalesceExec if !conf.enableColumnarCoalesce => fail(p) - case p: CartesianProductExec if !conf.cartesianProductTransformerEnabled => fail(p) + case p: SortExec if !glutenConf.enableColumnarSort => fail(p) + case p: WindowExec if !glutenConf.enableColumnarWindow => fail(p) + case p: SortMergeJoinExec if !glutenConf.enableColumnarSortMergeJoin => fail(p) + case p: BatchScanExec if !glutenConf.enableColumnarBatchScan => fail(p) + case p: FileSourceScanExec if !glutenConf.enableColumnarFileScan => fail(p) + case p: ProjectExec if !glutenConf.enableColumnarProject => fail(p) + case p: FilterExec if !glutenConf.enableColumnarFilter => fail(p) + case p: UnionExec if !glutenConf.enableColumnarUnion => fail(p) + case p: ExpandExec if !glutenConf.enableColumnarExpand => fail(p) + case p: SortAggregateExec if !glutenConf.forceToUseHashAgg => fail(p) + case p: ShuffledHashJoinExec if !glutenConf.enableColumnarShuffledHashJoin => fail(p) + case p: ShuffleExchangeExec if !glutenConf.enableColumnarShuffle => fail(p) + case p: BroadcastExchangeExec if !glutenConf.enableColumnarBroadcastExchange => fail(p) + case p @ (_: LocalLimitExec | _: GlobalLimitExec) if !glutenConf.enableColumnarLimit => + fail(p) + case p: GenerateExec if !glutenConf.enableColumnarGenerate => fail(p) + case p: CoalesceExec if !glutenConf.enableColumnarCoalesce => fail(p) + case p: CartesianProductExec if !glutenConf.cartesianProductTransformerEnabled => fail(p) case p: TakeOrderedAndProjectExec - if !(conf.enableTakeOrderedAndProject && conf.enableColumnarSort && - conf.enableColumnarShuffle && conf.enableColumnarProject) => + if !(glutenConf.enableTakeOrderedAndProject && glutenConf.enableColumnarSort && + glutenConf.enableColumnarShuffle && glutenConf.enableColumnarProject) => fail(p) - case p: BroadcastHashJoinExec if !conf.enableColumnarBroadcastJoin => + case p: BroadcastHashJoinExec if !glutenConf.enableColumnarBroadcastJoin => fail(p) case p: BroadcastNestedLoopJoinExec - if !(conf.enableColumnarBroadcastJoin && - conf.broadcastNestedLoopJoinTransformerTransformerEnabled) => + if !(glutenConf.enableColumnarBroadcastJoin && + glutenConf.broadcastNestedLoopJoinTransformerTransformerEnabled) => fail(p) case p @ (_: HashAggregateExec | _: SortAggregateExec | _: ObjectHashAggregateExec) - if !conf.enableColumnarHashAgg => + if !glutenConf.enableColumnarHashAgg => fail(p) case p if SparkShimLoader.getSparkShims.isWindowGroupLimitExec( - plan) && !conf.enableColumnarWindowGroupLimit => + plan) && !glutenConf.enableColumnarWindowGroupLimit => fail(p) case p - if HiveTableScanExecTransformer.isHiveTableScan(p) && !conf.enableColumnarHiveTableScan => + if HiveTableScanExecTransformer.isHiveTableScan( + p) && !glutenConf.enableColumnarHiveTableScan => fail(p) case p: SampleExec - if !(conf.enableColumnarSample && BackendsApiManager.getSettings.supportSampleExec()) => + if !(glutenConf.enableColumnarSample && BackendsApiManager.getSettings + .supportSampleExec()) => fail(p) case _ => pass() } @@ -204,21 +199,205 @@ object Validators { } } - private class ValidatorPipeline(validators: Seq[Validator]) extends Validator { + private class FallbackIfScanOnlyWithFilterPushed(scanOnly: Boolean) extends Validator { override def validate(plan: SparkPlan): Validator.OutCome = { - val init: Validator.OutCome = pass() - val finalOut = validators.foldLeft(init) { - case (out, validator) => - out match { - case Validator.Passed => validator.validate(plan) - case Validator.Failed(_) => out + if (!scanOnly) { + return pass() + } + // Scan-only mode + plan match { + case _: BatchScanExec => pass() + case _: FileSourceScanExec => pass() + case p if HiveTableScanExecTransformer.isHiveTableScan(p) => pass() + case filter: FilterExec => + val childIsScan = filter.child.isInstanceOf[FileSourceScanExec] || + filter.child.isInstanceOf[BatchScanExec] + if (childIsScan) { + pass() + } else { + fail(filter) } + case other => fail(other) } - finalOut } } - private object NoopValidator extends Validator { - override def validate(plan: SparkPlan): Validator.OutCome = pass() + private class FallbackByNativeValidation extends Validator with Logging { + override def validate(plan: SparkPlan): Validator.OutCome = plan match { + case plan: BatchScanExec => + val transformer = ScanTransformerFactory.createBatchScanTransformer(plan) + transformer.doValidate().toValidatorOutcome() + case plan: FileSourceScanExec => + val transformer = ScanTransformerFactory.createFileSourceScanTransformer(plan) + transformer.doValidate().toValidatorOutcome() + case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => + HiveTableScanExecTransformer(plan).doValidate().toValidatorOutcome() + case plan: ProjectExec => + val transformer = ProjectExecTransformer(plan.projectList, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: FilterExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(plan.condition, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: HashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: SortAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: ObjectHashAggregateExec => + val transformer = HashAggregateExecBaseTransformer.from(plan) + transformer.doValidate().toValidatorOutcome() + case plan: UnionExec => + val transformer = ColumnarUnionExec(plan.children) + transformer.doValidate().toValidatorOutcome() + case plan: ExpandExec => + val transformer = ExpandExecTransformer(plan.projections, plan.output, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: WriteFilesExec => + val transformer = WriteFilesExecTransformer( + plan.child, + plan.fileFormat, + plan.partitionColumns, + plan.bucketSpec, + plan.options, + plan.staticPartitions) + transformer.doValidate().toValidatorOutcome() + case plan: SortExec => + val transformer = + SortExecTransformer(plan.sortOrder, plan.global, plan.child, plan.testSpillFrequency) + transformer.doValidate().toValidatorOutcome() + case plan: ShuffleExchangeExec => + val transformer = ColumnarShuffleExchangeExec(plan, plan.child, plan.child.output) + transformer.doValidate().toValidatorOutcome() + case plan: ShuffledHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genShuffledHashJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + OffloadJoin.getShjBuildSide(plan), + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().toValidatorOutcome() + case plan: BroadcastExchangeExec => + val transformer = ColumnarBroadcastExchangeExec(plan.mode, plan.child) + transformer.doValidate().toValidatorOutcome() + case bhj: BroadcastHashJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastHashJoinExecTransformer( + bhj.leftKeys, + bhj.rightKeys, + bhj.joinType, + bhj.buildSide, + bhj.condition, + bhj.left, + bhj.right, + isNullAwareAntiJoin = bhj.isNullAwareAntiJoin) + transformer.doValidate().toValidatorOutcome() + case plan: SortMergeJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genSortMergeJoinExecTransformer( + plan.leftKeys, + plan.rightKeys, + plan.joinType, + plan.condition, + plan.left, + plan.right, + plan.isSkewJoin) + transformer.doValidate().toValidatorOutcome() + case plan: CartesianProductExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genCartesianProductExecTransformer(plan.left, plan.right, plan.condition) + transformer.doValidate().toValidatorOutcome() + case plan: BroadcastNestedLoopJoinExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance + .genBroadcastNestedLoopJoinExecTransformer( + plan.left, + plan.right, + plan.buildSide, + plan.joinType, + plan.condition) + transformer.doValidate().toValidatorOutcome() + case plan: WindowExec => + val transformer = WindowExecTransformer( + plan.windowExpression, + plan.partitionSpec, + plan.orderSpec, + plan.child) + transformer.doValidate().toValidatorOutcome() + case plan if SparkShimLoader.getSparkShims.isWindowGroupLimitExec(plan) => + val windowGroupLimitPlan = SparkShimLoader.getSparkShims + .getWindowGroupLimitExecShim(plan) + .asInstanceOf[WindowGroupLimitExecShim] + val transformer = WindowGroupLimitExecTransformer( + windowGroupLimitPlan.partitionSpec, + windowGroupLimitPlan.orderSpec, + windowGroupLimitPlan.rankLikeFunction, + windowGroupLimitPlan.limit, + windowGroupLimitPlan.mode, + windowGroupLimitPlan.child + ) + transformer.doValidate().toValidatorOutcome() + case plan: CoalesceExec => + ColumnarCoalesceExec(plan.numPartitions, plan.child) + .doValidate() + .toValidatorOutcome() + case plan: GlobalLimitExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromGlobalLimit(plan) + val transformer = LimitExecTransformer(plan.child, offset, limit) + transformer.doValidate().toValidatorOutcome() + case plan: LocalLimitExec => + val transformer = LimitExecTransformer(plan.child, 0L, plan.limit) + transformer.doValidate().toValidatorOutcome() + case plan: GenerateExec => + val transformer = BackendsApiManager.getSparkPlanExecApiInstance.genGenerateTransformer( + plan.generator, + plan.requiredChildOutput, + plan.outer, + plan.generatorOutput, + plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: BatchEvalPythonExec => + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().toValidatorOutcome() + case plan: ArrowEvalPythonExec => + // When backend doesn't support ColumnarArrow or colunmnar arrow configuration not + // enabled, we will try offloading through EvalPythonExecTransformer + if ( + !BackendsApiManager.getSettings.supportColumnarArrowUdf() || + !GlutenConfig.getConf.enableColumnarArrowUDF + ) { + // Both CH and Velox will try using backend's built-in functions for calculate + val transformer = EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, plan.child) + transformer.doValidate().toValidatorOutcome() + } + pass() + case plan: TakeOrderedAndProjectExec => + val (limit, offset) = + SparkShimLoader.getSparkShims.getLimitAndOffsetFromTopK(plan) + val transformer = TakeOrderedAndProjectExecTransformer( + limit, + plan.sortOrder, + plan.projectList, + plan.child, + offset) + transformer.doValidate().toValidatorOutcome() + case plan: SampleExec => + val transformer = + BackendsApiManager.getSparkPlanExecApiInstance.genSampleExecTransformer( + plan.lowerBound, + plan.upperBound, + plan.withReplacement, + plan.seed, + plan.child) + transformer.doValidate().toValidatorOutcome() + case _ => + // Currently we assume a plan to be offload-able by default. + pass() + } } } diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala deleted file mode 100644 index 0a58b9f69c12..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/GlutenCostModel.scala +++ /dev/null @@ -1,48 +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. - */ -package org.apache.gluten.planner.cost - -import org.apache.gluten.GlutenConfig -import org.apache.gluten.ras.CostModel - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.utils.ReflectionUtil - -object GlutenCostModel extends Logging { - def find(): CostModel[SparkPlan] = { - val aliases: Map[String, Class[_ <: CostModel[SparkPlan]]] = - Map( - "legacy" -> classOf[LegacyCostModel], - "rough" -> classOf[RoughCostModel], - "rough2" -> classOf[RoughCostModel2]) - val aliasOrClass = GlutenConfig.getConf.rasCostModel - val clazz: Class[_ <: CostModel[SparkPlan]] = if (aliases.contains(aliasOrClass)) { - aliases(aliasOrClass) - } else { - val userModel = ReflectionUtil.classForName(aliasOrClass) - logInfo(s"Using user cost model: $aliasOrClass") - userModel - } - val ctor = clazz.getDeclaredConstructor() - ctor.setAccessible(true) - val model = ctor.newInstance() - model - } - - def legacy(): CostModel[SparkPlan] = new LegacyCostModel() -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala b/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala deleted file mode 100644 index d3c8410abf88..000000000000 --- a/gluten-substrait/src/main/scala/org/apache/gluten/planner/cost/LongCostModel.scala +++ /dev/null @@ -1,55 +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. - */ -package org.apache.gluten.planner.cost - -import org.apache.gluten.planner.plan.GlutenPlanModel.GroupLeafExec -import org.apache.gluten.ras.{Cost, CostModel} - -import org.apache.spark.sql.execution.SparkPlan - -abstract class LongCostModel extends CostModel[SparkPlan] { - private val infLongCost = Long.MaxValue - - override def costOf(node: SparkPlan): GlutenCost = node match { - case _: GroupLeafExec => throw new IllegalStateException() - case _ => GlutenCost(longCostOf(node)) - } - - private def longCostOf(node: SparkPlan): Long = node match { - case n => - val selfCost = selfLongCostOf(n) - - // Sum with ceil to avoid overflow. - def safeSum(a: Long, b: Long): Long = { - assert(a >= 0) - assert(b >= 0) - val sum = a + b - if (sum < a || sum < b) Long.MaxValue else sum - } - - (n.children.map(longCostOf).toList :+ selfCost).reduce(safeSum) - } - - def selfLongCostOf(node: SparkPlan): Long - - override def costComparator(): Ordering[Cost] = Ordering.Long.on { - case GlutenCost(value) => value - case _ => throw new IllegalStateException("Unexpected cost type") - } - - override def makeInfCost(): Cost = GlutenCost(infLongCost) -} diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala index 79148d9f3093..1ceb2d4155ab 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/substrait/SubstraitContext.scala @@ -112,19 +112,6 @@ class SubstraitContext extends Serializable { id } - /** - * Register empty rel list to certain operator id. Used when the computing of a Spark transformer - * is omitted. - * @param operatorId - * operator id - */ - def registerEmptyRelToOperator(operatorId: JLong): Unit = { - if (!operatorToRelsMap.containsKey(operatorId)) { - val rels = new JArrayList[JLong]() - operatorToRelsMap.put(operatorId, rels) - } - } - /** * Return the registered map. * @return diff --git a/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala b/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala index 85be57493f02..e4fc11441031 100644 --- a/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala +++ b/gluten-substrait/src/main/scala/org/apache/gluten/utils/PullOutProjectHelper.scala @@ -62,7 +62,8 @@ trait PullOutProjectHelper { protected def replaceExpressionWithAttribute( expr: Expression, projectExprsMap: mutable.HashMap[Expression, NamedExpression], - replaceBoundReference: Boolean = false): Expression = + replaceBoundReference: Boolean = false, + replaceLiteral: Boolean = true): Expression = expr match { case alias: Alias => alias.child match { @@ -73,6 +74,7 @@ trait PullOutProjectHelper { } case attr: Attribute => attr case e: BoundReference if !replaceBoundReference => e + case literal: Literal if !replaceLiteral => literal case other => projectExprsMap .getOrElseUpdate(other.canonicalized, Alias(other, generatePreAliasName)()) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala index d55733fe4e97..1de490ad6165 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarBroadcastExchangeExec.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.ValidatablePlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.GlutenTimeMetric import org.apache.gluten.sql.shims.SparkShimLoader @@ -41,7 +42,7 @@ import scala.util.control.NonFatal case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) extends BroadcastExchangeLike - with GlutenPlan { + with ValidatablePlan { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics: Map[String, SQLMetric] = @@ -121,12 +122,14 @@ case class ColumnarBroadcastExchangeExec(mode: BroadcastMode, child: SparkPlan) @transient private val timeout: Long = SQLConf.get.broadcastTimeout - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override def doCanonicalize(): SparkPlan = { val canonicalized = BackendsApiManager.getSparkPlanExecApiInstance.doCanonicalizeForBroadcastMode(mode) diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala index ada7283da8d5..2bdc1cf4b371 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarCollapseTransformStages.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.backend.Backend import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution._ -import org.apache.gluten.extension.columnar.transition.Convention +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.SubstraitContext import org.apache.gluten.substrait.rel.RelBuilder @@ -32,6 +31,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.adaptive.BroadcastQueryStageExec +import org.apache.spark.sql.execution.exchange.BroadcastExchangeLike import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch @@ -49,14 +50,18 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp @transient override lazy val metrics: Map[String, SQLMetric] = - BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetrics(sparkContext) + BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetrics( + child, + sparkContext, + forBroadcast()) override def simpleString(maxFields: Int): String = { s"$nodeName${truncatedString(output, "[", ", ", "]", maxFields)}" } override def metricsUpdater(): MetricsUpdater = - BackendsApiManager.getMetricsApiInstance.genInputIteratorTransformerMetricsUpdater(metrics) + BackendsApiManager.getMetricsApiInstance + .genInputIteratorTransformerMetricsUpdater(metrics, forBroadcast()) override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning @@ -69,12 +74,20 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp override protected def doTransform(context: SubstraitContext): TransformContext = { val operatorId = context.nextOperatorId(nodeName) val readRel = RelBuilder.makeReadRelForInputIterator(child.output.asJava, context, operatorId) - TransformContext(output, output, readRel) + TransformContext(output, readRel) } override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { copy(child = newChild) } + + private def forBroadcast(): Boolean = { + child match { + case ColumnarInputAdapter(c) if c.isInstanceOf[BroadcastQueryStageExec] => true + case ColumnarInputAdapter(c) if c.isInstanceOf[BroadcastExchangeLike] => true + case _ => false + } + } } /** @@ -114,7 +127,7 @@ case class InputIteratorTransformer(child: SparkPlan) extends UnaryTransformSupp * generate/compile code. */ case class ColumnarCollapseTransformStages( - glutenConfig: GlutenConfig, + glutenConf: GlutenConfig, transformStageCounter: AtomicInteger = ColumnarCollapseTransformStages.transformStageCounter) extends Rule[SparkPlan] { @@ -160,13 +173,22 @@ case class ColumnarCollapseTransformStages( } } +// TODO: Make this inherit from GlutenPlan. case class ColumnarInputAdapter(child: SparkPlan) extends InputAdapterGenerateTreeStringShim - with Convention.KnownBatchType { + with Convention.KnownBatchType + with Convention.KnownRowTypeForSpark33OrLater + with GlutenPlan.SupportsRowBasedCompatible + with ConventionReq.KnownChildConvention { override def output: Seq[Attribute] = child.output - override def supportsColumnar: Boolean = true + final override val supportsColumnar: Boolean = true + final override val supportsRowBased: Boolean = false + override def rowType0(): Convention.RowType = Convention.RowType.None override def batchType(): Convention.BatchType = - Backend.get().defaultBatchType + BackendsApiManager.getSettings.primaryBatchType + override def requiredChildConvention(): Seq[ConventionReq] = Seq( + ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType))) override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override protected def doExecuteColumnar(): RDD[ColumnarBatch] = child.executeColumnar() override def outputPartitioning: Partitioning = child.outputPartitioning diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index 4f62377b09e3..007a93bd06a1 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.{GlutenPlan, ValidationResult} +import org.apache.gluten.execution.ValidatablePlan +import org.apache.gluten.extension.ValidationResult +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark._ @@ -45,7 +47,7 @@ case class ColumnarShuffleExchangeExec( projectOutputAttributes: Seq[Attribute], advisoryPartitionSize: Option[Long] = None) extends ShuffleExchangeLike - with GlutenPlan { + with ValidatablePlan { private[sql] lazy val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) @@ -126,7 +128,6 @@ case class ColumnarShuffleExchangeExec( override def nodeName: String = "ColumnarExchange" - override def supportsColumnar: Boolean = true override def numMappers: Int = shuffleDependency.rdd.getNumPartitions override def numPartitions: Int = shuffleDependency.partitioner.numPartitions @@ -149,6 +150,10 @@ case class ColumnarShuffleExchangeExec( super.stringArgs ++ Iterator(s"[shuffle_writer_type=$shuffleWriterType]") } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala index 2c1edd04bb4a..12280cc42aed 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarSubqueryBroadcastExec.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.transition.Convention import org.apache.gluten.metrics.GlutenTimeMetric import org.apache.spark.rdd.RDD @@ -26,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.joins.{BuildSideRelation, HashedRelation, HashJoin, LongHashedRelation} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.types.IntegralType import org.apache.spark.util.ThreadUtils import scala.concurrent.Future @@ -63,6 +65,14 @@ case class ColumnarSubqueryBroadcastExec( copy(name = "native-dpp", buildKeys = keys, child = child.canonicalized) } + // Copy from org.apache.spark.sql.execution.joins.HashJoin#canRewriteAsLongType + // we should keep consistent with it to identify the LongHashRelation. + private def canRewriteAsLongType(keys: Seq[Expression]): Boolean = { + // TODO: support BooleanType, DateType and TimestampType + keys.forall(_.dataType.isInstanceOf[IntegralType]) && + keys.map(_.dataType.defaultSize).sum <= 8 + } + @transient private lazy val relationFuture: Future[Array[InternalRow]] = { // relationFuture is used in "doExecute". Therefore we can get the execution id correctly here. @@ -77,7 +87,13 @@ case class ColumnarSubqueryBroadcastExec( relation match { case b: BuildSideRelation => // Transform columnar broadcast value to Array[InternalRow] by key. - b.transform(buildKeys(index)).distinct + if (canRewriteAsLongType(buildKeys)) { + b.transform(HashJoin.extractKeyExprAt(buildKeys, index)).distinct + } else { + b.transform( + BoundReference(index, buildKeys(index).dataType, buildKeys(index).nullable)) + .distinct + } case h: HashedRelation => val (iter, expr) = if (h.isInstanceOf[LongHashedRelation]) { (h.keys(), HashJoin.extractKeyExprAt(buildKeys, index)) @@ -107,6 +123,10 @@ case class ColumnarSubqueryBroadcastExec( relationFuture } + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.None + override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException( "SubqueryBroadcastExec does not support the execute() code path.") diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala index fcd82d8c1987..45f202637020 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/ColumnarWriteFilesExec.scala @@ -18,10 +18,9 @@ package org.apache.spark.sql.execution import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.exception.GlutenException -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Convention.{KnownRowType, RowType} -import org.apache.gluten.extension.columnar.transition.ConventionReq -import org.apache.gluten.extension.columnar.transition.ConventionReq.KnownChildrenConventions +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.extension.columnar.transition.{Convention, ConventionReq} +import org.apache.gluten.extension.columnar.transition.Convention.RowType import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.TaskContext @@ -45,18 +44,19 @@ abstract class ColumnarWriteFilesExec protected ( override val right: SparkPlan) extends BinaryExecNode with GlutenPlan - with KnownChildrenConventions - with KnownRowType with ColumnarWriteFilesExec.ExecuteWriteCompatible { val child: SparkPlan = left override lazy val references: AttributeSet = AttributeSet.empty - override def supportsColumnar: Boolean = true - - override def requiredChildrenConventions(): Seq[ConventionReq] = { - List(ConventionReq.backendBatch) + override def requiredChildConvention(): Seq[ConventionReq] = { + val req = ConventionReq.ofBatch( + ConventionReq.BatchType.Is(BackendsApiManager.getSettings.primaryBatchType)) + Seq.tabulate(2)( + _ => { + req + }) } /** @@ -69,7 +69,8 @@ abstract class ColumnarWriteFilesExec protected ( * * Since https://github.com/apache/incubator-gluten/pull/6745. */ - override def rowType(): RowType = { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): RowType = { RowType.VanillaRow } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala index fa697789c8cf..7a3335052423 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala @@ -16,8 +16,7 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.execution.WholeStageTransformer -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, WholeStageTransformer} import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.utils.PlanUtil @@ -147,15 +146,13 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { } } - // spotless:off - // scalastyle:off + // format: off /** * Given a input physical plan, performs the following tasks. * 1. Generates the explain output for the input plan excluding the subquery plans. 2. Generates * the explain output for each subquery referenced in the plan. */ - // scalastyle:on - // spotless:on + // format: on def processPlan[T <: QueryPlan[T]]( plan: T, append: String => Unit, diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala index f6e23e7cff67..c28395941b0c 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenFallbackReporter.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig import org.apache.gluten.events.GlutenPlanFallbackEvent -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.logging.LogLevelUtil @@ -31,12 +31,12 @@ import org.apache.spark.sql.execution.ui.GlutenEventUtils * This rule is used to collect all fallback reason. * 1. print fallback reason for each plan node 2. post all fallback reason using one event */ -case class GlutenFallbackReporter(glutenConfig: GlutenConfig, spark: SparkSession) +case class GlutenFallbackReporter(glutenConf: GlutenConfig, spark: SparkSession) extends Rule[SparkPlan] with LogLevelUtil { override def apply(plan: SparkPlan): SparkPlan = { - if (!glutenConfig.enableFallbackReport) { + if (!glutenConf.enableFallbackReport) { return plan } printFallbackReason(plan) @@ -52,7 +52,7 @@ case class GlutenFallbackReporter(glutenConfig: GlutenConfig, spark: SparkSessio } private def printFallbackReason(plan: SparkPlan): Unit = { - val validationLogLevel = glutenConfig.validationLogLevel + val validationLogLevel = glutenConf.validationLogLevel plan.foreachUp { case _: GlutenPlan => // ignore case p: SparkPlan if FallbackTags.nonEmpty(p) => diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala index 4ecc674d4b13..709673feab0b 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala @@ -16,16 +16,14 @@ */ package org.apache.spark.sql.execution -import org.apache.gluten.execution.WholeStageTransformer -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{GlutenPlan, WholeStageTransformer} import org.apache.gluten.utils.PlanUtil - import org.apache.spark.sql.{AnalysisException, Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{CommandResult, LogicalPlan} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.execution.ColumnarWriteFilesExec.NoopLeaf -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AQEShuffleReadExec, AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedCommandExec} import org.apache.spark.sql.execution.datasources.WriteFilesExec @@ -36,7 +34,8 @@ import org.apache.spark.sql.internal.SQLConf import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -// spotless:off + +// format: off /** * A helper class to get the Gluten fallback summary from a Spark [[Dataset]]. * @@ -52,7 +51,7 @@ import scala.collection.mutable.ArrayBuffer * df.fallbackSummary * }}} */ -// spotless:on +// format: on object GlutenImplicits { case class FallbackSummary( diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala index 9ec75aa209d1..fc0cd39ecfa8 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenFormatWriterInjectsBase.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql.execution.datasources import org.apache.gluten.execution.{ProjectExecTransformer, SortExecTransformer, TransformSupport, WholeStageTransformer} import org.apache.gluten.execution.datasource.GlutenFormatWriterInjects -import org.apache.gluten.extension.columnar.AddFallbackTagRule -import org.apache.gluten.extension.columnar.MiscColumnarRules.TransformPreOverrides -import org.apache.gluten.extension.columnar.rewrite.RewriteSparkPlanRulesManager +import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -28,6 +26,7 @@ import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, SparkPla import org.apache.spark.sql.execution.ColumnarCollapseTransformStages.transformStageCounter trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { + private lazy val transform = HeuristicTransform.static() /** * FileFormatWriter wraps some Project & Sort on the top of the original output spark plan, we @@ -42,12 +41,9 @@ trait GlutenFormatWriterInjectsBase extends GlutenFormatWriterInjects { return plan.execute() } - val rules = List( - RewriteSparkPlanRulesManager(), - AddFallbackTagRule(), - TransformPreOverrides() - ) - val transformed = rules.foldLeft(plan) { case (latestPlan, rule) => rule.apply(latestPlan) } + // FIXME: HeuristicTransform is costly. Re-applying it may cause performance issues. + val transformed = transform(plan) + if (!transformed.isInstanceOf[TransformSupport]) { throw new IllegalStateException( "Cannot transform the SparkPlans wrapped by FileFormatWriter, " + diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala index d33e779eb332..126417bf18a5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/datasources/GlutenWriterColumnarRules.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.execution.datasources import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.execution.ColumnarToRowExecBase +import org.apache.gluten.execution.{ColumnarToRowExecBase, GlutenPlan} import org.apache.gluten.execution.datasource.GlutenFormatFactory -import org.apache.gluten.extension.GlutenPlan -import org.apache.gluten.extension.columnar.transition.Transitions +import org.apache.gluten.extension.columnar.transition.{Convention, Transitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -61,6 +60,10 @@ case class FakeRowAdaptor(child: SparkPlan) override def output: Seq[Attribute] = child.output + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + + override def rowType0(): Convention.RowType = Convention.RowType.VanillaRow + override protected def doExecute(): RDD[InternalRow] = { doExecuteColumnar().map(cb => new FakeRow(cb)) } @@ -74,7 +77,7 @@ case class FakeRowAdaptor(child: SparkPlan) if (child.supportsColumnar) { child.executeColumnar() } else { - val r2c = Transitions.toBackendBatchPlan(child) + val r2c = Transitions.toBatchPlan(child, BackendsApiManager.getSettings.primaryBatchType) r2c.executeColumnar() } } @@ -121,8 +124,10 @@ object GlutenWriterColumnarRules { command.table.storage.outputFormat .flatMap(formatMapping.get) .filter(GlutenFormatFactory.isRegistered) - case _: CreateHiveTableAsSelectCommand => - None + case command: CreateHiveTableAsSelectCommand => + command.tableDesc.storage.outputFormat + .flatMap(formatMapping.get) + .filter(GlutenFormatFactory.isRegistered) case _ => None } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala index 60f3e2ffd966..e9dbeb560c68 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/joins/BuildSideRelation.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.joins import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode import org.apache.spark.sql.vectorized.ColumnarBatch trait BuildSideRelation extends Serializable { @@ -26,11 +27,19 @@ trait BuildSideRelation extends Serializable { def deserialized: Iterator[ColumnarBatch] /** - * Transform columnar broadcasted value to Array[InternalRow] by key and distinct. + * Transform columnar broadcasted value to Array[InternalRow] by key. * @return */ def transform(key: Expression): Array[InternalRow] /** Returns a read-only copy of this, to be safely used in current thread. */ def asReadOnlyCopy(): BuildSideRelation + + /** + * The broadcast mode that is associated with this relation in Gluten allows for direct + * broadcasting of the original relation, so transforming a relation has a post-processing nature. + * + * Post-processed relation transforms can use this mode to obtain the desired format. + */ + val mode: BroadcastMode } diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala index 6a9da0a9cf92..7b4c09d4f9e5 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala @@ -99,7 +99,7 @@ case class EvalPythonExecTransformer( val relNode = getRelNode(childCtx.root, expressionNodes, context, operatorId, child.output, false) - TransformContext(child.output, output, relNode) + TransformContext(output, relNode) } def getRelNode( diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala index 85432350d4a2..6911672376a9 100644 --- a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.ValidationResult import org.apache.gluten.metrics.MetricsUpdater import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat @@ -45,7 +44,8 @@ import java.net.URI case class HiveTableScanExecTransformer( requestedAttributes: Seq[Attribute], relation: HiveTableRelation, - partitionPruningPred: Seq[Expression])(@transient session: SparkSession) + partitionPruningPred: Seq[Expression], + prunedOutput: Seq[Attribute] = Seq.empty[Attribute])(@transient session: SparkSession) extends AbstractHiveTableScanExec(requestedAttributes, relation, partitionPruningPred)(session) with BasicScanExecTransformer { @@ -63,7 +63,13 @@ case class HiveTableScanExecTransformer( override def getMetadataColumns(): Seq[AttributeReference] = Seq.empty - override def outputAttributes(): Seq[Attribute] = output + override def outputAttributes(): Seq[Attribute] = { + if (prunedOutput.nonEmpty) { + prunedOutput + } else { + output + } + } override def getPartitions: Seq[InputPartition] = partitions @@ -174,8 +180,8 @@ case class HiveTableScanExecTransformer( object HiveTableScanExecTransformer { - val NULL_VALUE: Char = 0x00 - val DEFAULT_FIELD_DELIMITER: Char = 0x01 + private val NULL_VALUE: Char = 0x00 + private val DEFAULT_FIELD_DELIMITER: Char = 0x01 val TEXT_INPUT_FORMAT_CLASS: Class[TextInputFormat] = Utils.classForName("org.apache.hadoop.mapred.TextInputFormat") val ORC_INPUT_FORMAT_CLASS: Class[OrcInputFormat] = @@ -186,24 +192,6 @@ object HiveTableScanExecTransformer { plan.isInstanceOf[HiveTableScanExec] } - def copyWith(plan: SparkPlan, newPartitionFilters: Seq[Expression]): SparkPlan = { - val hiveTableScanExec = plan.asInstanceOf[HiveTableScanExec] - hiveTableScanExec.copy(partitionPruningPred = newPartitionFilters)(sparkSession = - hiveTableScanExec.session) - } - - def validate(plan: SparkPlan): ValidationResult = { - plan match { - case hiveTableScan: HiveTableScanExec => - val hiveTableScanTransformer = new HiveTableScanExecTransformer( - hiveTableScan.requestedAttributes, - hiveTableScan.relation, - hiveTableScan.partitionPruningPred)(hiveTableScan.session) - hiveTableScanTransformer.doValidate() - case _ => ValidationResult.failed("Is not a Hive scan") - } - } - def apply(plan: SparkPlan): HiveTableScanExecTransformer = { plan match { case hiveTableScan: HiveTableScanExec => diff --git a/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala new file mode 100644 index 000000000000..70b08a2ed8bb --- /dev/null +++ b/gluten-substrait/src/main/scala/org/apache/spark/sql/hive/HiveTableScanNestedColumnPruning.scala @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.execution.ProjectExecTransformer + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.catalog.HiveTableRelation +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, SparkPlan} +import org.apache.spark.sql.hive.HiveTableScanExecTransformer.{ORC_INPUT_FORMAT_CLASS, PARQUET_INPUT_FORMAT_CLASS, TEXT_INPUT_FORMAT_CLASS} +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.util.SchemaUtils._ +import org.apache.spark.util.Utils + +object HiveTableScanNestedColumnPruning extends Logging { + import org.apache.spark.sql.catalyst.expressions.SchemaPruning._ + + def supportNestedColumnPruning(project: ProjectExecTransformer): Boolean = { + if (GlutenConfig.getConf.enableColumnarHiveTableScanNestedColumnPruning) { + project.child match { + case HiveTableScanExecTransformer(_, relation, _, _) => + relation.tableMeta.storage.inputFormat match { + case Some(inputFormat) + if TEXT_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + relation.tableMeta.storage.serde match { + case Some("org.openx.data.jsonserde.JsonSerDe") | Some( + "org.apache.hive.hcatalog.data.JsonSerDe") => + return true + case _ => + } + case Some(inputFormat) + if ORC_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case Some(inputFormat) + if PARQUET_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + return true + case _ => + } + case _ => + } + } + false + } + + def apply(plan: SparkPlan): SparkPlan = { + plan match { + case ProjectExecTransformer(projectList, child) => + child match { + case h: HiveTableScanExecTransformer => + val newPlan = prunePhysicalColumns( + h.relation, + projectList, + Seq.empty[Expression], + (prunedDataSchema, prunedMetadataSchema) => { + buildNewHiveTableScan(h, prunedDataSchema, prunedMetadataSchema) + }, + (schema, requestFields) => { + h.pruneSchema(schema, requestFields) + } + ) + if (newPlan.nonEmpty) { + return newPlan.get + } else { + return ProjectExecTransformer(projectList, child) + } + case _ => + return ProjectExecTransformer(projectList, child) + } + case _ => + } + plan + } + + private def prunePhysicalColumns( + relation: HiveTableRelation, + projects: Seq[NamedExpression], + filters: Seq[Expression], + leafNodeBuilder: (StructType, StructType) => LeafExecNode, + pruneSchemaFunc: (StructType, Seq[SchemaPruning.RootField]) => StructType) + : Option[SparkPlan] = { + val (normalizedProjects, normalizedFilters) = + normalizeAttributeRefNames(relation.output, projects, filters) + val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) + // If requestedRootFields includes a nested field, continue. Otherwise, + // return op + if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { + val prunedDataSchema = pruneSchemaFunc(relation.tableMeta.dataSchema, requestedRootFields) + val metaFieldNames = relation.tableMeta.schema.fieldNames + val metadataSchema = relation.output.collect { + case attr: AttributeReference if metaFieldNames.contains(attr.name) => attr + }.toStructType + val prunedMetadataSchema = if (metadataSchema.nonEmpty) { + pruneSchemaFunc(metadataSchema, requestedRootFields) + } else { + metadataSchema + } + // If the data schema is different from the pruned data schema + // OR + // the metadata schema is different from the pruned metadata schema, continue. + // Otherwise, return None. + if ( + countLeaves(relation.tableMeta.dataSchema) > countLeaves(prunedDataSchema) || + countLeaves(metadataSchema) > countLeaves(prunedMetadataSchema) + ) { + val leafNode = leafNodeBuilder(prunedDataSchema, prunedMetadataSchema) + val projectionOverSchema = ProjectionOverSchema( + prunedDataSchema.merge(prunedMetadataSchema), + AttributeSet(relation.output)) + Some( + buildNewProjection( + projects, + normalizedProjects, + normalizedFilters, + leafNode, + projectionOverSchema)) + } else { + None + } + } else { + None + } + } + + /** + * Normalizes the names of the attribute references in the given projects and filters to reflect + * the names in the given logical relation. This makes it possible to compare attributes and + * fields by name. Returns a tuple with the normalized projects and filters, respectively. + */ + private def normalizeAttributeRefNames( + output: Seq[AttributeReference], + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { + val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap + val normalizedProjects = projects + .map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + .map { case expr: NamedExpression => expr } + val normalizedFilters = filters.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + (normalizedProjects, normalizedFilters) + } + + /** Builds the new output [[Project]] Spark SQL operator that has the `leafNode`. */ + private def buildNewProjection( + projects: Seq[NamedExpression], + normalizedProjects: Seq[NamedExpression], + filters: Seq[Expression], + leafNode: LeafExecNode, + projectionOverSchema: ProjectionOverSchema): ProjectExecTransformer = { + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + FilterExec(newFilterCondition, leafNode) + } else { + leafNode + } + + // Construct the new projections of our Project by + // rewriting the original projections + val newProjects = + normalizedProjects.map(_.transformDown { case projectionOverSchema(expr) => expr }).map { + case expr: NamedExpression => expr + } + + ProjectExecTransformer( + restoreOriginalOutputNames(newProjects, projects.map(_.name)), + projectionChild) + } + + private def buildNewHiveTableScan( + hiveTableScan: HiveTableScanExecTransformer, + prunedDataSchema: StructType, + prunedMetadataSchema: StructType): HiveTableScanExecTransformer = { + val relation = hiveTableScan.relation + val partitionSchema = relation.tableMeta.partitionSchema + val prunedBaseSchema = StructType( + prunedDataSchema.fields.filterNot( + f => partitionSchema.fieldNames.contains(f.name)) ++ partitionSchema.fields) + val finalSchema = prunedBaseSchema.merge(prunedMetadataSchema) + val prunedOutput = getPrunedOutput(relation.output, finalSchema) + var finalOutput = Seq.empty[Attribute] + for (p <- hiveTableScan.output) { + var flag = false + for (q <- prunedOutput if !flag) { + if (p.name.equals(q.name)) { + finalOutput :+= q + flag = true + } + } + } + HiveTableScanExecTransformer( + hiveTableScan.requestedAttributes, + relation, + hiveTableScan.partitionPruningPred, + finalOutput)(hiveTableScan.session) + } + + // Prune the given output to make it consistent with `requiredSchema`. + private def getPrunedOutput( + output: Seq[AttributeReference], + requiredSchema: StructType): Seq[Attribute] = { + // We need to update the data type of the output attributes to use the pruned ones. + // so that references to the original relation's output are not broken + val nameAttributeMap = output.map(att => (att.name, att)).toMap + val requiredAttributes = + requiredSchema.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + requiredAttributes.map { + case att if nameAttributeMap.contains(att.name) => + nameAttributeMap(att.name).withDataType(att.dataType) + case att => att + } + } + + /** + * Counts the "leaf" fields of the given dataType. Informally, this is the number of fields of + * non-complex data type in the tree representation of [[DataType]]. + */ + private def countLeaves(dataType: DataType): Int = { + dataType match { + case array: ArrayType => countLeaves(array.elementType) + case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) + case struct: StructType => + struct.map(field => countLeaves(field.dataType)).sum + case _ => 1 + } + } +} diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala index a27a4991cf4a..ce2f85af7781 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/benchmarks/RandomParquetDataGenerator.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.types._ import com.github.javafaker.Faker -import java.sql.Date +import java.sql.{Date, Timestamp} import java.util.Random case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { @@ -67,7 +67,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { case DoubleType => faker.number().randomDouble(2, Double.MinValue.toLong, Double.MaxValue.toLong) case DateType => new Date(faker.date().birthday().getTime) -// case TimestampType => new Timestamp(faker.date().birthday().getTime) + case TimestampType => new Timestamp(faker.date().birthday().getTime) case t: DecimalType => BigDecimal( faker.number().randomDouble(t.scale, 0, Math.pow(10, t.precision - t.scale).toLong)) @@ -124,7 +124,7 @@ case class RandomParquetDataGenerator(initialSeed: Long = 0L) extends Logging { () => StructField(fieldName, FloatType, nullable = true), () => StructField(fieldName, DoubleType, nullable = true), () => StructField(fieldName, DateType, nullable = true), -// () => StructField(fieldName, TimestampType, nullable = true), + () => StructField(fieldName, TimestampType, nullable = true), () => StructField(fieldName, DecimalType(10, 2), nullable = true), () => StructField(fieldName, DecimalType(30, 10), nullable = true) ) diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala index 146d6fde58ec..08081fadb5f9 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/execution/WholeStageTransformerSuite.scala @@ -17,15 +17,13 @@ package org.apache.gluten.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan import org.apache.gluten.test.FallbackUtil import org.apache.gluten.utils.Arm import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, GlutenQueryTest, Row} -import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.DoubleType @@ -33,7 +31,6 @@ import java.io.File import java.util.concurrent.atomic.AtomicBoolean import scala.io.Source -import scala.reflect.ClassTag case class Table(name: String, partitionColumns: Seq[String]) @@ -179,128 +176,39 @@ abstract class WholeStageTransformerSuite result } - def checkLengthAndPlan(df: DataFrame, len: Int = 100): Unit = { - assert(df.collect().length == len) - val executedPlan = getExecutedPlan(df) - assert(executedPlan.exists(plan => plan.find(_.isInstanceOf[TransformSupport]).isDefined)) - } - - /** - * Get all the children plan of plans. - * @param plans: - * the input plans. - * @return - */ - def getChildrenPlan(plans: Seq[SparkPlan]): Seq[SparkPlan] = { - if (plans.isEmpty) { - return Seq() - } - - val inputPlans: Seq[SparkPlan] = plans.map { - case stage: ShuffleQueryStageExec => stage.plan - case plan => plan - } - - var newChildren: Seq[SparkPlan] = Seq() - inputPlans.foreach { - plan => - newChildren = newChildren ++ getChildrenPlan(plan.children) - // To avoid duplication of WholeStageCodegenXXX and its children. - if (!plan.nodeName.startsWith("WholeStageCodegen")) { - newChildren = newChildren :+ plan - } - } - newChildren - } - - /** - * Get the executed plan of a data frame. - * @param df: - * dataframe. - * @return - * A sequence of executed plans. - */ - def getExecutedPlan(df: DataFrame): Seq[SparkPlan] = { - df.queryExecution.executedPlan match { - case exec: AdaptiveSparkPlanExec => - getChildrenPlan(Seq(exec.executedPlan)) - case cmd: CommandResultExec => - getChildrenPlan(Seq(cmd.commandPhysicalPlan)) - case plan => - getChildrenPlan(Seq(plan)) - } - } - - /** - * Check whether the executed plan of a dataframe contains the expected plan. - * @param df: - * the input dataframe. - * @param tag: - * class of the expected plan. - * @tparam T: - * type of the expected plan. - */ - def checkGlutenOperatorMatch[T <: GlutenPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - val executedPlan = getExecutedPlan(df) - assert( - executedPlan.exists(plan => tag.runtimeClass.isInstance(plan)), - s"Expect ${tag.runtimeClass.getSimpleName} exists " + - s"in executedPlan:\n ${executedPlan.last}" - ) - } - - def checkSparkOperatorMatch[T <: SparkPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { - val executedPlan = getExecutedPlan(df) - assert(executedPlan.exists(plan => tag.runtimeClass.isInstance(plan))) - } - - /** - * Check whether the executed plan of a dataframe contains the expected plan chain. - * - * @param df - * : the input dataframe. - * @param tag - * : class of the expected plan. - * @param childTag - * : class of the expected plan's child. - * @tparam T - * : type of the expected plan. - * @tparam PT - * : type of the expected plan's child. - */ - def checkSparkOperatorChainMatch[T <: UnaryExecNode, PT <: UnaryExecNode]( - df: DataFrame)(implicit tag: ClassTag[T], childTag: ClassTag[PT]): Unit = { - val executedPlan = getExecutedPlan(df) - assert( - executedPlan.exists( - plan => - tag.runtimeClass.isInstance(plan) - && childTag.runtimeClass.isInstance(plan.children.head)), - s"Expect an operator chain of [${tag.runtimeClass.getSimpleName} ->" - + s"${childTag.runtimeClass.getSimpleName}] exists in executedPlan: \n" - + s"${executedPlan.last}" - ) + protected def compareResultsAgainstVanillaSpark( + sql: String, + compareResult: Boolean = true, + customCheck: DataFrame => Unit, + noFallBack: Boolean = true, + cache: Boolean = false): DataFrame = { + compareDfResultsAgainstVanillaSpark( + () => spark.sql(sql), + compareResult, + customCheck, + noFallBack, + cache) } /** * run a query with native engine as well as vanilla spark then compare the result set for * correctness check */ - protected def compareResultsAgainstVanillaSpark( - sqlStr: String, + protected def compareDfResultsAgainstVanillaSpark( + dataframe: () => DataFrame, compareResult: Boolean = true, customCheck: DataFrame => Unit, noFallBack: Boolean = true, cache: Boolean = false): DataFrame = { var expected: Seq[Row] = null withSQLConf(vanillaSparkConfs(): _*) { - val df = spark.sql(sqlStr) + val df = dataframe() expected = df.collect() } - // By default we will fallabck complex type scan but here we should allow + // By default, we will fallback complex type scan but here we should allow // to test support of complex type spark.conf.set("spark.gluten.sql.complexType.scan.fallback.enabled", "false"); - val df = spark.sql(sqlStr) + val df = dataframe() if (cache) { df.cache() } @@ -345,7 +253,12 @@ abstract class WholeStageTransformerSuite noFallBack: Boolean = true, cache: Boolean = false)(customCheck: DataFrame => Unit): DataFrame = { - compareResultsAgainstVanillaSpark(sqlStr, compareResult, customCheck, noFallBack, cache) + compareDfResultsAgainstVanillaSpark( + () => spark.sql(sqlStr), + compareResult, + customCheck, + noFallBack, + cache) } /** @@ -362,8 +275,8 @@ abstract class WholeStageTransformerSuite customCheck: DataFrame => Unit, noFallBack: Boolean = true, compareResult: Boolean = true): Unit = - compareResultsAgainstVanillaSpark( - tpchSQL(queryNum, tpchQueries), + compareDfResultsAgainstVanillaSpark( + () => spark.sql(tpchSQL(queryNum, tpchQueries)), compareResult, customCheck, noFallBack) diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala index 5c35cb50207a..fec36ac1acfa 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuite.scala @@ -17,8 +17,7 @@ package org.apache.gluten.extension.columnar.transition import org.apache.gluten.exception.GlutenException -import org.apache.gluten.execution.ColumnarToColumnarExec -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{ColumnarToColumnarExec, GlutenPlan} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -32,7 +31,7 @@ class TransitionSuite extends SharedSparkSession { test("Trivial C2R") { val in = BatchLeaf(TypeA) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchLeaf(TypeA))) } @@ -40,7 +39,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R") { val in = RowUnary(BatchLeaf(TypeA)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == RowUnary(BatchToRow(TypeA, BatchLeaf(TypeA)))) } @@ -48,7 +47,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert R2C") { val in = BatchUnary(TypeA, RowLeaf()) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert(out == BatchToRow(TypeA, BatchUnary(TypeA, RowToBatch(TypeA, RowLeaf())))) } @@ -56,7 +55,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2R2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeB)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -67,7 +66,7 @@ class TransitionSuite extends SharedSparkSession { test("Insert C2C") { val in = BatchUnary(TypeA, BatchLeaf(TypeC)) val out = ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } assert( out == BatchToRow( @@ -79,7 +78,7 @@ class TransitionSuite extends SharedSparkSession { val in = BatchUnary(TypeA, BatchLeaf(TypeD)) assertThrows[GlutenException] { ConventionFunc.ignoreBackend { - Transitions.insertTransitions(in, outputsColumnar = false) + Transitions.insert(in, outputsColumnar = false) } } } @@ -87,29 +86,37 @@ class TransitionSuite extends SharedSparkSession { object TransitionSuite extends TransitionSuiteBase { object TypeA extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + } } object TypeB extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + } } object TypeC extends Convention.BatchType { - fromRow(RowToBatch(this, _)) - toRow(BatchToRow(this, _)) - fromBatch(TypeA, BatchToBatch(TypeA, this, _)) - toBatch(TypeA, BatchToBatch(this, TypeA, _)) + override protected[this] def registerTransitions(): Unit = { + fromRow(RowToBatch(this, _)) + toRow(BatchToRow(this, _)) + fromBatch(TypeA, BatchToBatch(TypeA, this, _)) + toBatch(TypeA, BatchToBatch(this, TypeA, _)) + } } - object TypeD extends Convention.BatchType {} + object TypeD extends Convention.BatchType { + override protected[this] def registerTransitions(): Unit = {} + } case class RowToBatch(toBatchType: Convention.BatchType, override val child: SparkPlan) extends RowToColumnarTransition with GlutenPlan { - override def supportsColumnar: Boolean = true - override protected def batchType0(): Convention.BatchType = toBatchType + override def batchType(): Convention.BatchType = toBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) override protected def doExecute(): RDD[InternalRow] = diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala index d82cc3aac9f1..7ab9b5d0cfaf 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/extension/columnar/transition/TransitionSuiteBase.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.extension.columnar.transition -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -24,22 +24,20 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.{BinaryExecNode, LeafExecNode, SparkPlan, UnaryExecNode} trait TransitionSuiteBase { - case class BatchLeaf(override val batchType0: Convention.BatchType) + case class BatchLeaf(override val batchType: Convention.BatchType) extends LeafExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = List.empty } - case class BatchUnary( - override val batchType0: Convention.BatchType, - override val child: SparkPlan) + case class BatchUnary(override val batchType: Convention.BatchType, override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = copy(child = newChild) @@ -50,12 +48,12 @@ trait TransitionSuiteBase { } case class BatchBinary( - override val batchType0: Convention.BatchType, + override val batchType: Convention.BatchType, override val left: SparkPlan, override val right: SparkPlan) extends BinaryExecNode with GlutenPlan { - override def supportsColumnar: Boolean = true + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def withNewChildrenInternal( newLeft: SparkPlan, diff --git a/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala b/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala index 3d26dd16c4eb..2a0ecaf92ca9 100644 --- a/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala +++ b/gluten-substrait/src/test/scala/org/apache/gluten/test/FallbackUtil.scala @@ -16,7 +16,7 @@ */ package org.apache.gluten.test -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ diff --git a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala index 53abaa9ac297..8507233a57fc 100644 --- a/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala +++ b/gluten-substrait/src/test/scala/org/apache/spark/sql/GlutenQueryTest.scala @@ -21,6 +21,8 @@ package org.apache.spark.sql * 1. We need to modify the way org.apache.spark.sql.CHQueryTest#compare compares double */ import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.GlutenPlan +import org.apache.gluten.execution.TransformSupport import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.SPARK_VERSION_SHORT @@ -28,7 +30,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.{CommandResultExec, SparkPlan, SQLExecution, UnaryExecNode} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.storage.StorageLevel @@ -38,6 +41,7 @@ import org.scalatest.Assertions import java.util.TimeZone import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import scala.reflect.runtime.universe abstract class GlutenQueryTest extends PlanTest { @@ -306,6 +310,109 @@ abstract class GlutenQueryTest extends PlanTest { query.queryExecution.executedPlan.missingInput.isEmpty, s"The physical plan has missing inputs:\n${query.queryExecution.executedPlan}") } + + def checkLengthAndPlan(df: DataFrame, len: Int = 100): Unit = { + assert(df.collect().length == len) + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.find(_.isInstanceOf[TransformSupport]).isDefined)) + } + + /** + * Get all the children plan of plans. + * @param plans: + * the input plans. + * @return + */ + def getChildrenPlan(plans: Seq[SparkPlan]): Seq[SparkPlan] = { + if (plans.isEmpty) { + return Seq() + } + + val inputPlans: Seq[SparkPlan] = plans.map { + case stage: ShuffleQueryStageExec => stage.plan + case plan => plan + } + + var newChildren: Seq[SparkPlan] = Seq() + inputPlans.foreach { + plan => + newChildren = newChildren ++ getChildrenPlan(plan.children) + // To avoid duplication of WholeStageCodegenXXX and its children. + if (!plan.nodeName.startsWith("WholeStageCodegen")) { + newChildren = newChildren :+ plan + } + } + newChildren + } + + /** + * Get the executed plan of a data frame. + * @param df: + * dataframe. + * @return + * A sequence of executed plans. + */ + def getExecutedPlan(df: DataFrame): Seq[SparkPlan] = { + df.queryExecution.executedPlan match { + case exec: AdaptiveSparkPlanExec => + getChildrenPlan(Seq(exec.executedPlan)) + case cmd: CommandResultExec => + getChildrenPlan(Seq(cmd.commandPhysicalPlan)) + case plan => + getChildrenPlan(Seq(plan)) + } + } + + /** + * Check whether the executed plan of a dataframe contains the expected plan chain. + * + * @param df + * : the input dataframe. + * @param tag + * : class of the expected plan. + * @param childTag + * : class of the expected plan's child. + * @tparam T + * : type of the expected plan. + * @tparam PT + * : type of the expected plan's child. + */ + def checkSparkOperatorChainMatch[T <: UnaryExecNode, PT <: UnaryExecNode]( + df: DataFrame)(implicit tag: ClassTag[T], childTag: ClassTag[PT]): Unit = { + val executedPlan = getExecutedPlan(df) + assert( + executedPlan.exists( + plan => + tag.runtimeClass.isInstance(plan) + && childTag.runtimeClass.isInstance(plan.children.head)), + s"Expect an operator chain of [${tag.runtimeClass.getSimpleName} ->" + + s"${childTag.runtimeClass.getSimpleName}] exists in executedPlan: \n" + + s"${executedPlan.last}" + ) + } + + /** + * Check whether the executed plan of a dataframe contains the expected plan. + * @param df: + * the input dataframe. + * @param tag: + * class of the expected plan. + * @tparam T: + * type of the expected plan. + */ + def checkGlutenOperatorMatch[T <: GlutenPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert( + executedPlan.exists(plan => tag.runtimeClass.isInstance(plan)), + s"Expect ${tag.runtimeClass.getSimpleName} exists " + + s"in executedPlan:\n ${executedPlan.last}" + ) + } + + def checkSparkOperatorMatch[T <: SparkPlan](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => tag.runtimeClass.isInstance(plan))) + } } object GlutenQueryTest extends Assertions { diff --git a/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala b/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala index 25150ca31f9f..a2f608f64d2e 100644 --- a/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala +++ b/gluten-ui/src/main/scala/org/apache/spark/sql/execution/ui/GlutenSQLAppStatusListener.scala @@ -32,7 +32,7 @@ private class GlutenSQLAppStatusListener(conf: SparkConf, kvstore: ElementTracki private val executionIdToDescription = new mutable.HashMap[Long, String] private val executionIdToFallbackEvent = new mutable.HashMap[Long, GlutenPlanFallbackEvent] - kvstore.addTrigger(classOf[SQLExecutionUIData], conf.get[Int](UI_RETAINED_EXECUTIONS)) { + kvstore.addTrigger(classOf[GlutenSQLExecutionUIData], conf.get[Int](UI_RETAINED_EXECUTIONS)) { count => cleanupExecutions(count) } @@ -71,7 +71,7 @@ private class GlutenSQLAppStatusListener(conf: SparkConf, kvstore: ElementTracki fallbackEvent.get.physicalPlanDescription, fallbackEvent.get.fallbackNodeToReason.toSeq.sortBy(_._1) ) - kvstore.write(uiData) + kvstore.write(uiData, checkTriggers = true) executionIdToFallbackEvent.remove(event.executionId) } executionIdToDescription.put(event.executionId, event.description) diff --git a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java index 84fac8ace697..d32da8d93b31 100644 --- a/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java +++ b/gluten-uniffle/velox/src/main/java/org/apache/spark/shuffle/writer/VeloxUniffleColumnarShuffleWriter.java @@ -17,6 +17,7 @@ package org.apache.spark.shuffle.writer; import org.apache.gluten.GlutenConfig; +import org.apache.gluten.backendsapi.BackendsApiManager; import org.apache.gluten.columnarbatch.ColumnarBatches; import org.apache.gluten.memory.memtarget.MemoryTarget; import org.apache.gluten.memory.memtarget.Spiller; @@ -70,7 +71,8 @@ public class VeloxUniffleColumnarShuffleWriter extends RssShuffleWriter> records) { if (cb.numRows() == 0 || cb.numCols() == 0) { LOG.info("Skip ColumnarBatch of 0 rows or 0 cols"); } else { - long handle = ColumnarBatches.getNativeHandle(cb); + long handle = ColumnarBatches.getNativeHandle(BackendsApiManager.getBackendName(), cb); if (nativeShuffleWriter == -1) { nativeShuffleWriter = jniWrapper.makeForRSS( diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 27e26606f653..36d5b5177c6b 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -203,6 +203,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude( "SPARK-32376: Make unionByName null-filling behavior work with struct columns - deep expr") .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple .exclude("except all") .exclude("exceptAll - nullability") .exclude("intersectAll") @@ -551,8 +554,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") @@ -844,8 +845,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -854,8 +853,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK-34814: LikeSimplification should handle NULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 55fb4ae16d1e..15495270a189 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -318,9 +318,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenSortSuite] // Sort spill is not supported. .exclude("sorting does not crash for large inputs") @@ -367,6 +364,22 @@ class VeloxTestSettings extends BackendTestSettings { // Rewrite the following two tests in GlutenDatasetSuite. .exclude("dropDuplicates: columns with same column name") .exclude("groupBy.as") + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") @@ -861,7 +874,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") // Rewrite for supported INT96 - timestamp. @@ -875,8 +887,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. .exclude("Filter applied on merged Parquet schema with new column should work") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index fc31289119a1..754c29241864 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, RemoveFallbackTagRule} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.RemoveFallbackTagRule +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +45,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +62,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +80,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +100,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +120,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -199,19 +200,19 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index 51204b0777d6..2f690c615556 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -106,54 +105,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query - // should work without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 88d771ec8a4f..8b5efa91673f 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -60,7 +60,7 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" override def doCanonicalize(): TestFileSourceScanExecTransformer = { TestFileSourceScanExecTransformer( diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index 7a3c4d1056e7..e525eb1a9c01 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index da950e2fc1ee..b9bf4e1ac40f 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -225,6 +225,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple .exclude("except all") .exclude("exceptAll - nullability") .exclude("intersectAll") @@ -567,8 +570,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") @@ -817,8 +818,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -827,8 +826,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 8b56f63f65df..407b9c8b95cc 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenAnsiCastSuiteWithAnsiModeOff, GlutenAnsiCastSuiteWithAnsiModeOn, GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCastSuiteWithAnsiModeOn, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryCastSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector._ import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution._ @@ -142,6 +142,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") @@ -663,7 +679,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -679,11 +694,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -699,8 +711,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Support Parquet column index") .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] @@ -710,13 +720,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-35640: int as long should throw schema incompatible error") // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -899,9 +902,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 3e776721311c..88e0ecf65a9a 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,19 +229,19 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index f394b4687d3d..33bf1a1ec97e 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution import org.apache.gluten.execution.HashAggregateExecBaseTransformer -import org.apache.gluten.utils.BackendTestUtils import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, SortAggregateExec} @@ -85,7 +84,7 @@ class GlutenReplaceHashWithSortAggSuite withTempView("t1", "t2") { spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { aggExprInfo => val query = s""" @@ -99,11 +98,7 @@ class GlutenReplaceHashWithSortAggSuite |) |GROUP BY key """.stripMargin - if (BackendTestUtils.isCHBackendLoaded()) { - checkAggs(query, 1, 0, 1, 0) - } else { - checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) - } + checkAggs(query, aggExprInfo._2, aggExprInfo._3, aggExprInfo._4, aggExprInfo._5) } } } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index a1163f9525b4..02b30a46a63c 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -106,54 +105,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala index c2446e38d75f..4a73a8dc72de 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -21,6 +21,7 @@ import org.apache.gluten.events.GlutenPlanFallbackEvent import org.apache.gluten.execution.FileSourceScanExecTransformer import org.apache.gluten.utils.BackendTestUtils +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} import org.apache.spark.sql.execution.ProjectExec @@ -32,6 +33,10 @@ import org.apache.spark.status.ElementTrackingStore import scala.collection.mutable.ArrayBuffer class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + } testGluten("test fallback logging") { val testAppender = new LogAppender("fallback reason") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index 40874cd69d30..34e97273d603 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index ac08fc5a80cc..a407c5d68247 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -227,6 +227,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple enableSuite[GlutenDataFrameStatSuite] enableSuite[GlutenDataFrameSuite] .exclude("Uuid expressions should produce same results at retries in the same DataFrame") @@ -510,8 +513,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") @@ -740,8 +741,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -750,8 +749,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") @@ -847,7 +844,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("do not replace hash aggregate if child does not have sort order") .exclude("do not replace hash aggregate if there is no group-by column") .excludeGlutenTest("replace partial hash aggregate with sort aggregate") - .excludeGlutenTest("replace partial and final hash aggregate together with sort aggregate") enableSuite[GlutenReuseExchangeAndSubquerySuite] enableSuite[GlutenSQLAggregateFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] diff --git a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 22a9e62c09ae..dbb01fbe7067 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryEvalSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector.{GlutenDataSourceV2DataFrameSessionCatalogSuite, GlutenDataSourceV2DataFrameSuite, GlutenDataSourceV2FunctionSuite, GlutenDataSourceV2SQLSessionCatalogSuite, GlutenDataSourceV2SQLSuiteV1Filter, GlutenDataSourceV2SQLSuiteV2Filter, GlutenDataSourceV2Suite, GlutenDeleteFromTableSuite, GlutenDeltaBasedDeleteFromTableSuite, GlutenFileDataSourceV2FallBackSuite, GlutenGroupBasedDeleteFromTableSuite, GlutenKeyGroupedPartitioningSuite, GlutenLocalScanSuite, GlutenMetadataColumnSuite, GlutenSupportsCatalogOptionsSuite, GlutenTableCapabilityCheckSuite, GlutenWriteDistributionAndOrderingSuite} import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution.{FallbackStrategiesSuite, GlutenBroadcastExchangeSuite, GlutenCoalesceShufflePartitionsSuite, GlutenExchangeSuite, GlutenLocalBroadcastExchangeSuite, GlutenReplaceHashWithSortAggSuite, GlutenReuseExchangeAndSubquerySuite, GlutenSameResultSuite, GlutenSortSuite, GlutenSQLAggregateFunctionSuite, GlutenSQLWindowFunctionSuite, GlutenTakeOrderedAndProjectSuite} @@ -120,6 +120,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // Velox does not support single quotes in get_json_object function. .exclude("function get_json_object - support single quotes") @@ -644,7 +660,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -660,11 +675,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -680,8 +692,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] @@ -694,13 +704,6 @@ class VeloxTestSettings extends BackendTestSettings { // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -899,9 +902,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet.enableSuite[GlutenSQLWindowFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] @@ -969,6 +969,13 @@ class VeloxTestSettings extends BackendTestSettings { // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") enableSuite[GlutenFileSourceCharVarcharTestSuite] + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in map key") + .exclude("length check for input string values: nested in map value") + .exclude("length check for input string values: nested in both map key and value") + .exclude("length check for input string values: nested in array of struct") + .exclude("length check for input string values: nested in array of array") enableSuite[GlutenDSV2CharVarcharTestSuite] enableSuite[GlutenColumnExpressionSuite] // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala index 84502ace5110..89d9114870b6 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -16,8 +16,107 @@ */ package org.apache.spark.sql +import org.apache.spark.SparkException class GlutenFileSourceCharVarcharTestSuite extends FileSourceCharVarcharTestSuite - with GlutenSQLTestsTrait {} + with GlutenSQLTestsTrait { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + private val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + testGluten("length check for input string values: nested in struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT) USING $format") + sql("INSERT INTO t SELECT struct(null)") + checkAnswer(spark.table("t"), Row(Row(null))) + val e = intercept[RuntimeException] { + sql("INSERT INTO t SELECT struct('123456')") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map key") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), STRING>) USING $format") + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP) USING $format") + sql("INSERT INTO t VALUES (map('a', null))") + checkAnswer(spark.table("t"), Row(Map("a" -> null))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in both map key and value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), $typeName(5)>) USING $format") + val e1 = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e1.getMessage.contains(ERROR_MESSAGE)) + val e2 = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e2.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in struct of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT>) USING $format") + sql("INSERT INTO t SELECT struct(array(null))") + checkAnswer(spark.table("t"), Row(Row(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t SELECT struct(array('123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(struct(null)))") + checkAnswer(spark.table("t"), Row(Seq(Row(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(struct('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(array(null)))") + checkAnswer(spark.table("t"), Row(Seq(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(array('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } +} class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index 3e776721311c..88e0ecf65a9a 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -43,7 +44,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -60,7 +61,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -78,7 +79,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -98,7 +99,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -118,7 +119,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -228,19 +229,19 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index 332c21418a9b..33bf1a1ec97e 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -84,7 +84,7 @@ class GlutenReplaceHashWithSortAggSuite withTempView("t1", "t2") { spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { aggExprInfo => val query = s""" diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala index 640b8cdc6f8f..cdfb63aa2074 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.benchmarks import org.apache.gluten.GlutenConfig +import org.apache.gluten.backendsapi.BackendsApiManager import org.apache.gluten.execution.{FileSourceScanExecTransformer, WholeStageTransformer} import org.apache.gluten.extension.columnar.transition.Transitions import org.apache.gluten.jni.JniLibLoader @@ -125,7 +126,7 @@ object ParquetReadBenchmark extends SqlBasedBenchmark { // generate ColumnarToRow val columnarToRowPlan = - Transitions.toBackendBatchPlan(newWholeStage) + Transitions.toBatchPlan(newWholeStage, BackendsApiManager.getSettings.primaryBatchType) val newWholeStageRDD = newWholeStage.executeColumnar() val newColumnarToRowRDD = columnarToRowPlan.execute() diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index b4a4b6017b67..a4f830e18716 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -105,54 +104,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala index 74c4df197759..e6cc2937a43e 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources import org.apache.gluten.execution.{ProjectExecTransformer, SortExecTransformer} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.executor.OutputMetrics diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index e5e122c9065d..f7cc114859cd 100644 --- a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala index 9e4c81081de1..9c22af0434af 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/clickhouse/ClickHouseTestSettings.scala @@ -227,6 +227,9 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35756: unionByName support struct having same col names but different sequence") .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + .exclude( + "SPARK-36673: Only merge nullability for Unions of struct" + ) // disabled due to case-insensitive not supported in CH tuple enableSuite[GlutenDataFrameStatSuite] enableSuite[GlutenDataFrameSuite] .exclude("Uuid expressions should produce same results at retries in the same DataFrame") @@ -510,8 +513,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-17617: % (Remainder) double % double on super big double") .exclude("Abs") .exclude("pmod") - .exclude("function least") - .exclude("function greatest") .exclude("SPARK-28322: IntegralDivide supports decimal type") .exclude("SPARK-33008: division by zero on divide-like operations returns incorrect result") .exclude("SPARK-34920: error class") @@ -740,8 +741,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32110: compare special double/float values in struct") enableSuite[GlutenRandomSuite].exclude("random").exclude("SPARK-9127 codegen with long seed") enableSuite[GlutenRegexpExpressionsSuite] - .exclude("LIKE ALL") - .exclude("LIKE ANY") .exclude("LIKE Pattern") .exclude("LIKE Pattern ESCAPE '/'") .exclude("LIKE Pattern ESCAPE '#'") @@ -750,8 +749,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("RegexReplace") .exclude("RegexExtract") .exclude("RegexExtractAll") - .exclude("SPLIT") - .exclude("SPARK - 34814: LikeSimplification should handleNULL") enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") enableSuite[GlutenStringExpressionsSuite] .exclude("StringComparison") @@ -847,7 +844,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("do not replace hash aggregate if child does not have sort order") .exclude("do not replace hash aggregate if there is no group-by column") .excludeGlutenTest("replace partial hash aggregate with sort aggregate") - .excludeGlutenTest("replace partial and final hash aggregate together with sort aggregate") enableSuite[GlutenReuseExchangeAndSubquerySuite] enableSuite[GlutenSQLAggregateFunctionSuite] enableSuite[GlutenSQLWindowFunctionSuite] diff --git a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala index 3f6bea5dd1ce..f5a1a076956e 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/gluten/utils/velox/VeloxTestSettings.scala @@ -20,7 +20,7 @@ import org.apache.gluten.utils.{BackendTestSettings, SQLQueryTestSettings} import org.apache.spark.GlutenSortShuffleSuite import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenDecimalPrecisionSuite, GlutenHashExpressionsSuite, GlutenHigherOrderFunctionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite, GlutenTryEvalSuite} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.connector._ import org.apache.spark.sql.errors.{GlutenQueryCompilationErrorsDSv2Suite, GlutenQueryCompilationErrorsSuite, GlutenQueryExecutionErrorsSuite, GlutenQueryParsingErrorsSuite} import org.apache.spark.sql.execution._ @@ -121,6 +121,22 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenHigherOrderFunctionsSuite] enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonExpressionsSuite] + // https://github.com/apache/incubator-gluten/issues/8102 + .exclude("$.store.book") + .exclude("$") + .exclude("$.store.book[0]") + .exclude("$.store.book[*]") + .exclude("$.store.book[*].category") + .exclude("$.store.book[*].isbn") + .exclude("$.store.book[*].reader") + .exclude("$.store.basket[*]") + .exclude("$.store.basket[*][0]") + .exclude("$.store.basket[0][*]") + .exclude("$.store.basket[*][*]") + .exclude("$.store.basket[0][*].b") + // Exception class different. + .exclude("from_json - invalid data") enableSuite[GlutenJsonFunctionsSuite] // * in get_json_object expression not supported in velox .exclude("SPARK-42782: Hive compatibility check for get_json_object") @@ -655,7 +671,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenParquetFileFormatV2Suite] enableSuite[GlutenParquetV1FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -671,11 +686,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetV2FilterSuite] // Rewrite. - .exclude("Filter applied on merged Parquet schema with new column should work") .exclude("SPARK-23852: Broken Parquet push-down for partially-written stats") // Rewrite for supported INT96 - timestamp. .exclude("filter pushdown - timestamp") @@ -691,8 +703,6 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34562: Bloom filter push down") .exclude("SPARK-16371 Do not push down filters when inner name and outer name are the same") .exclude("filter pushdown - StringPredicate") - // https://github.com/apache/incubator-gluten/issues/7174 - .excludeGlutenTest("Filter applied on merged Parquet schema with new column should work") enableSuite[GlutenParquetInteroperabilitySuite] .exclude("parquet timestamp conversion") enableSuite[GlutenParquetIOSuite] @@ -705,13 +715,6 @@ class VeloxTestSettings extends BackendTestSettings { // Velox only support read Timestamp with INT96 for now. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") - // TODO: Unsupported Array schema in Parquet. - .exclude("vectorized reader: optional array with required elements") - .exclude("vectorized reader: required array with required elements") - .exclude("vectorized reader: required array with optional elements") - .exclude("vectorized reader: required array with legacy format") - // add support in native reader - .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") enableSuite[GlutenParquetV1PartitionDiscoverySuite] enableSuite[GlutenParquetV2PartitionDiscoverySuite] @@ -913,9 +916,6 @@ class VeloxTestSettings extends BackendTestSettings { // spill not supported yet. enableSuite[GlutenSQLWindowFunctionSuite] .exclude("test with low buffer spill threshold") - // https://github.com/apache/incubator-gluten/issues/7631 - .exclude( - "SPARK-16633: lead/lag should return the default value if the offset row does not exist") enableSuite[GlutenTakeOrderedAndProjectSuite] enableSuite[GlutenSessionExtensionSuite] enableSuite[TestFileSourceScanExecTransformer] @@ -989,6 +989,13 @@ class VeloxTestSettings extends BackendTestSettings { // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") enableSuite[GlutenFileSourceCharVarcharTestSuite] + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in array") + .exclude("length check for input string values: nested in map key") + .exclude("length check for input string values: nested in map value") + .exclude("length check for input string values: nested in both map key and value") + .exclude("length check for input string values: nested in array of struct") + .exclude("length check for input string values: nested in array of array") enableSuite[GlutenDSV2CharVarcharTestSuite] enableSuite[GlutenColumnExpressionSuite] // Velox raise_error('errMsg') throws a velox_user_error exception with the message 'errMsg'. diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala index 84502ace5110..ce2f1b465e7f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -16,8 +16,108 @@ */ package org.apache.spark.sql +import org.apache.spark.SparkException + class GlutenFileSourceCharVarcharTestSuite extends FileSourceCharVarcharTestSuite - with GlutenSQLTestsTrait {} + with GlutenSQLTestsTrait { + private def testTableWrite(f: String => Unit): Unit = { + withTable("t")(f("char")) + withTable("t")(f("varchar")) + } + + private val ERROR_MESSAGE = + "Exceeds char/varchar type length limitation: 5" + + testGluten("length check for input string values: nested in struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT) USING $format") + sql("INSERT INTO t SELECT struct(null)") + checkAnswer(spark.table("t"), Row(Row(null))) + val e = intercept[RuntimeException] { + sql("INSERT INTO t SELECT struct('123456')") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY<$typeName(5)>) USING $format") + sql("INSERT INTO t VALUES (array(null))") + checkAnswer(spark.table("t"), Row(Seq(null))) + val e = intercept[SparkException] { + sql("INSERT INTO t VALUES (array('a', '123456'))") + } + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map key") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), STRING>) USING $format") + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in map value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP) USING $format") + sql("INSERT INTO t VALUES (map('a', null))") + checkAnswer(spark.table("t"), Row(Map("a" -> null))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in both map key and value") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c MAP<$typeName(5), $typeName(5)>) USING $format") + val e1 = intercept[SparkException](sql("INSERT INTO t VALUES (map('123456', 'a'))")) + assert(e1.getMessage.contains(ERROR_MESSAGE)) + val e2 = intercept[SparkException](sql("INSERT INTO t VALUES (map('a', '123456'))")) + assert(e2.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in struct of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c STRUCT>) USING $format") + sql("INSERT INTO t SELECT struct(array(null))") + checkAnswer(spark.table("t"), Row(Row(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t SELECT struct(array('123456'))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of struct") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(struct(null)))") + checkAnswer(spark.table("t"), Row(Seq(Row(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(struct('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } + + testGluten("length check for input string values: nested in array of array") { + testTableWrite { + typeName => + sql(s"CREATE TABLE t(c ARRAY>) USING $format") + sql("INSERT INTO t VALUES (array(array(null)))") + checkAnswer(spark.table("t"), Row(Seq(Seq(null)))) + val e = intercept[SparkException](sql("INSERT INTO t VALUES (array(array('123456')))")) + assert(e.getMessage.contains(ERROR_MESSAGE)) + } + } +} class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala index 4008f862e17d..3b2db7117f4a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql +import org.apache.gluten.GlutenConfig import org.apache.gluten.execution.{ProjectExecTransformer, WholeStageTransformer} import org.apache.spark.SparkException @@ -323,41 +324,52 @@ class GlutenDataFrameSuite extends DataFrameSuite with GlutenSQLTestsTrait { } testGluten("Allow leading/trailing whitespace in string before casting") { - def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { - checkAnswer(df, expectedResult) - assert(find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) - } + withSQLConf(GlutenConfig.CAST_FROM_VARCHAR_ADD_TRIM_NODE.key -> "true") { + def checkResult(df: DataFrame, expectedResult: Seq[Row]): Unit = { + checkAnswer(df, expectedResult) + assert( + find(df.queryExecution.executedPlan)(_.isInstanceOf[ProjectExecTransformer]).isDefined) + } - // scalastyle:off nonascii - Seq(" 123", "123 ", " 123 ", "\u2000123\n\n\n", "123\r\r\r", "123\f\f\f", "123\u000C") - .toDF("col1") - .createOrReplaceTempView("t1") - // scalastyle:on nonascii - val expectedIntResult = Row(123) :: Row(123) :: - Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil - var df = spark.sql("select cast(col1 as int) from t1") - checkResult(df, expectedIntResult) - df = spark.sql("select cast(col1 as long) from t1") - checkResult(df, expectedIntResult) - - Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") - .toDF("col1") - .createOrReplaceTempView("t1") - val expectedFloatResult = Row(123.5) :: Row(123.5) :: - Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil - df = spark.sql("select cast(col1 as float) from t1") - checkResult(df, expectedFloatResult) - df = spark.sql("select cast(col1 as double) from t1") - checkResult(df, expectedFloatResult) - - // scalastyle:off nonascii - val rawData = - Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") - // scalastyle:on nonascii - rawData.toDF("col1").createOrReplaceTempView("t1") - val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq - df = spark.sql("select cast(col1 as binary) from t1") - checkResult(df, expectedBinaryResult) + // scalastyle:off nonascii + Seq( + " 123", + "123 ", + " 123 ", + "\u2000123\n\n\n", + "123\r\r\r", + "123\f\f\f", + "123\u000C", + "123\u0000") + .toDF("col1") + .createOrReplaceTempView("t1") + // scalastyle:on nonascii + val expectedIntResult = Row(123) :: Row(123) :: + Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Row(123) :: Nil + var df = spark.sql("select cast(col1 as int) from t1") + checkResult(df, expectedIntResult) + df = spark.sql("select cast(col1 as long) from t1") + checkResult(df, expectedIntResult) + + Seq(" 123.5", "123.5 ", " 123.5 ", "123.5\n\n\n", "123.5\r\r\r", "123.5\f\f\f", "123.5\u000C") + .toDF("col1") + .createOrReplaceTempView("t1") + val expectedFloatResult = Row(123.5) :: Row(123.5) :: + Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Row(123.5) :: Nil + df = spark.sql("select cast(col1 as float) from t1") + checkResult(df, expectedFloatResult) + df = spark.sql("select cast(col1 as double) from t1") + checkResult(df, expectedFloatResult) + + // scalastyle:off nonascii + val rawData = + Seq(" abc", "abc ", " abc ", "\u2000abc\n\n\n", "abc\r\r\r", "abc\f\f\f", "abc\u000C") + // scalastyle:on nonascii + rawData.toDF("col1").createOrReplaceTempView("t1") + val expectedBinaryResult = rawData.map(d => Row(d.getBytes(StandardCharsets.UTF_8))).seq + df = spark.sql("select cast(col1 as binary) from t1") + checkResult(df, expectedBinaryResult) + } } private def withExpr(newExpr: Expression): Column = new Column(newExpr) diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala index a214d9755e69..8908047a3321 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.sql.execution import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.BasicScanExecTransformer -import org.apache.gluten.extension.{GlutenPlan, GlutenSessionExtensions} -import org.apache.gluten.extension.columnar.{ExpandFallbackPolicy, FallbackTags, RemoveFallbackTagRule} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{BasicScanExecTransformer, GlutenPlan} +import org.apache.gluten.extension.GlutenSessionExtensions +import org.apache.gluten.extension.columnar.{FallbackTags, RemoveFallbackTagRule} import org.apache.gluten.extension.columnar.ColumnarRuleApplier.ColumnarRuleCall import org.apache.gluten.extension.columnar.MiscColumnarRules.RemoveTopmostColumnarToRow -import org.apache.gluten.extension.columnar.heuristic.HeuristicApplier -import org.apache.gluten.extension.columnar.transition.InsertTransitions +import org.apache.gluten.extension.columnar.heuristic.{ExpandFallbackPolicy, HeuristicApplier} +import org.apache.gluten.extension.columnar.transition.{Convention, InsertBackendTransitions} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{GlutenSQLTestsTrait, SparkSession} @@ -44,7 +45,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. assert(outputPlan == originalPlan) @@ -61,7 +62,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -79,7 +80,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -99,7 +100,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to fall back the entire plan. @@ -119,7 +120,7 @@ class FallbackStrategiesSuite extends GlutenSQLTestsTrait { _ => { UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) }, - c => InsertTransitions(c.outputsColumnar))) + c => InsertBackendTransitions(c.outputsColumnar))) .enableAdaptiveContext() val outputPlan = rule.apply(originalPlan, false) // Expect to get the plan with columnar rule applied. @@ -229,19 +230,19 @@ private object FallbackStrategiesSuite { } // For replacing LeafOp. - case class LeafOpTransformer(override val supportsColumnar: Boolean = true) - extends LeafExecNode - with GlutenPlan { + case class LeafOpTransformer() extends LeafExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = Seq.empty } // For replacing UnaryOp1. - case class UnaryOp1Transformer( - override val child: SparkPlan, - override val supportsColumnar: Boolean = true) + case class UnaryOp1Transformer(override val child: SparkPlan) extends UnaryExecNode with GlutenPlan { + override def batchType(): Convention.BatchType = BackendsApiManager.getSettings.primaryBatchType + override def rowType0(): Convention.RowType = Convention.RowType.None override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() override def output: Seq[Attribute] = child.output override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala index 332c21418a9b..33bf1a1ec97e 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -84,7 +84,7 @@ class GlutenReplaceHashWithSortAggSuite withTempView("t1", "t2") { spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") - Seq(("COUNT", 0, 1, 2, 0), ("COLLECT_LIST", 2, 0, 2, 0)).foreach { + Seq(("COUNT", 1, 0, 1, 0), ("COLLECT_LIST", 1, 0, 1, 0)).foreach { aggExprInfo => val query = s""" diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala index 9e4d94e1c298..bd1c269843fb 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -16,6 +16,27 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} -class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait { + testGluten("Parquet writer with ARRAY and MAP") { + spark.sql(""" + |CREATE TABLE T1 ( + | a INT, + | b ARRAY, + | c MAP + |) + |USING PARQUET + |""".stripMargin) + + spark.sql(""" + | INSERT OVERWRITE T1 VALUES + | (1, ARRAY(1, 2, 3), MAP("key1","value1")) + |""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM T1"), + Row(1, Array("1", "2", "3"), Map("key1" -> "value1")) :: Nil + ) + } +} diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala index 063b424e0d13..3c52ec82e9fc 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.LegacyBehaviorPolicy.{CORRECTED, LEGACY} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 @@ -105,54 +104,6 @@ abstract class GlutenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - testGluten("Filter applied on merged Parquet schema with new column should work") { - import testImplicits._ - withAllParquetReaders { - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> "true") { - withTempPath { - dir => - val path1 = s"${dir.getCanonicalPath}/table1" - (1 to 3) - .map(i => (i, i.toString, null: String)) - .toDF("a", "b", "c") - .write - .parquet(path1) - val path2 = s"${dir.getCanonicalPath}/table2" - (1 to 3) - .map(i => (null: Integer, i.toString, i.toString)) - .toDF("a", "b", "c") - .write - .parquet(path2) - - // No matter "c = 1" gets pushed down or not, this query should work without exception. - val df = spark.read.parquet(path1, path2).filter("c = 1").selectExpr("c", "b", "a") - df.show() - - // Annotated for the type check fails. - // checkAnswer(df, Row(1, "1", null)) - - val path3 = s"${dir.getCanonicalPath}/table3" - val dfStruct = sparkContext.parallelize(Seq((1, 1, null))).toDF("a", "b", "c") - dfStruct.select(struct("a").as("s")).write.parquet(path3) - - val path4 = s"${dir.getCanonicalPath}/table4" - val dfStruct2 = sparkContext.parallelize(Seq((null, 1, 1))).toDF("a", "b", "c") - dfStruct2.select(struct("c").as("s")).write.parquet(path4) - - // No matter "s.c = 1" gets pushed down or not, this query should work - // without exception. - val dfStruct3 = spark.read - .parquet(path3, path4) - .filter("s.c = 1") - .selectExpr("s") - checkAnswer(dfStruct3, Row(Row(null, 1))) - } - } - } - } - testGluten("SPARK-12218: 'Not' is included in Parquet filter pushdown") { import testImplicits._ diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala index d31936211b72..8d9e82df195a 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenInnerJoinSuiteForceShjOn extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenInnerJoinSuiteForceShjOff extends InnerJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala index 9c45eae15b0a..f48939c3d012 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -24,13 +24,13 @@ import org.apache.spark.sql.GlutenSQLTestsBaseTrait class GlutenOuterJoinSuiteForceShjOn extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "true") } } class GlutenOuterJoinSuiteForceShjOff extends OuterJoinSuite with GlutenSQLTestsBaseTrait { override def sparkConf: SparkConf = { super.sparkConf - .set(GlutenConfig.COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") + .set(GlutenConfig.COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED.key, "false") } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala index 5e6c66265ad7..d08b2837a599 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -45,7 +45,7 @@ class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { case f: TestFileSourceScanExecTransformer => f } assert(!testFileSourceScanExecTransformer.isEmpty) - assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestFile")) } } } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala index 85af9a623853..98ef4eff7869 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.scala @@ -58,5 +58,5 @@ case class TestFileSourceScanExecTransformer( optionalNumCoalescedBuckets, disableBucketedScan) - override val nodeNamePrefix: String = "TestNativeFile" + override val nodeNamePrefix: String = "TestFile" } diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala index d7ec1f0fa57e..7855f289a707 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -21,6 +21,7 @@ import org.apache.gluten.events.GlutenPlanFallbackEvent import org.apache.gluten.execution.FileSourceScanExecTransformer import org.apache.gluten.utils.BackendTestUtils +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql.{GlutenSQLTestsTrait, Row} import org.apache.spark.sql.execution.ProjectExec @@ -32,6 +33,10 @@ import org.apache.spark.status.ElementTrackingStore import scala.collection.mutable.ArrayBuffer class GlutenFallbackSuite extends GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.RAS_ENABLED.key, "false") + } testGluten("test fallback logging") { val testAppender = new LogAppender("fallback reason") diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala index 1cb905e10abf..38e032aec33f 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.sources import org.apache.gluten.GlutenColumnarWriteTestSupport +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.execution.SortExecTransformer -import org.apache.gluten.extension.GlutenPlan import org.apache.spark.SparkConf import org.apache.spark.executor.OutputMetrics diff --git a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala index e5e122c9065d..f7cc114859cd 100644 --- a/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala +++ b/gluten-ut/spark35/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.statistics import org.apache.gluten.GlutenConfig -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.GlutenPlan import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} diff --git a/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala new file mode 100644 index 000000000000..2aaf1be21bf3 --- /dev/null +++ b/gluten-ut/test/src/test/scala/org/apache/gluten/execution/MergeTwoPhasesHashBaseAggregateSuite.scala @@ -0,0 +1,202 @@ +/* + * 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.gluten.execution + +import org.apache.gluten.GlutenConfig +import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} + +abstract class BaseMergeTwoPhasesHashBaseAggregateSuite extends WholeStageTransformerSuite { + val fileFormat: String = "parquet" + override protected val resourcePath: String = "/tpch-data-parquet" + + override def beforeAll(): Unit = { + super.beforeAll() + + spark + .sql(s""" + |CREATE TABLE t1 (id int, age int, phone int, date string) + |USING $fileFormat + |PARTITIONED BY (date) + |""".stripMargin) + .show() + + spark + .sql(s""" + |INSERT INTO t1 PARTITION(date = '2020-01-01') + |SELECT id, id % 10 as age, id % 10 as phone + |FROM range(100) + |""".stripMargin) + .show() + } + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.sql.files.maxPartitionBytes", "1g") + .set("spark.sql.shuffle.partitions", "1") + .set("spark.memory.offHeap.size", "2g") + .set("spark.gluten.sql.mergeTwoPhasesAggregate.enabled", "true") + if (BackendTestUtils.isCHBackendLoaded()) { + conf + .set("spark.gluten.sql.enable.native.validation", "false") + .set(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + } + conf + } + + override def afterAll(): Unit = { + try { + sql("DROP TABLE IF EXISTS t1") + } finally { + super.afterAll() + } + } + + test("Merge two phase hash-based aggregate into one aggregate") { + def checkHashAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange hash aggregate, merge to one hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT key, count(key) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT key, count(key) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange hash aggregate + checkHashAggregateCount( + spark.sql(""" + |SELECT count(1) FROM t1 + |""".stripMargin), + 2) + } + + test("Merge two phase object-based aggregate into one aggregate") { + def checkObjectAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange object aggregate, merge to one hash aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT key, collect_list(key) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter object aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT key, collect_list(key) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange object aggregate + checkObjectAggregateCount( + spark.sql(""" + |SELECT collect_list(id) FROM t1 + |""".stripMargin), + 2) + } + + test("Merge two phase sort-based aggregate into one aggregate") { + def checkSortAggregateCount(df: DataFrame, expectedCount: Int): Unit = { + df.collect() + val plans = collect(df.queryExecution.executedPlan) { + case agg: HashAggregateExecBaseTransformer => agg + } + assert(plans.size == expectedCount) + } + + withSQLConf("spark.sql.test.forceApplySortAggregate" -> "true") { + withTempView("v1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("v1") + // no exchange sort aggregate, merge to one hash aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT sum(if(key<0,0,key)) + |FROM v1 + |GROUP BY key + |""".stripMargin), + 1 + ) + + // with filter sort aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT key, sum(if(key<0,0,key)) FILTER (WHERE key LIKE '%1%') AS pc2 + |FROM v1 + |GROUP BY key + |""".stripMargin), + 2 + ) + } + + // with exchange sort aggregate + checkSortAggregateCount( + spark.sql(""" + |SELECT sum(if(id<0,0,id)) FROM t1 + |""".stripMargin), + 2) + } + } +} + +class MergeTwoPhasesAggregateSuiteAEOn + extends BaseMergeTwoPhasesHashBaseAggregateSuite + with EnableAdaptiveExecutionSuite + +class MergeTwoPhasesAggregateSuiteAEOff + extends BaseMergeTwoPhasesHashBaseAggregateSuite + with DisableAdaptiveExecutionSuite diff --git a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala index d2a9611471ab..2088c90c019a 100644 --- a/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala +++ b/gluten-ut/test/src/test/scala/org/apache/spark/sql/GlutenExpressionDataTypesValidation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql import org.apache.gluten.GlutenConfig -import org.apache.gluten.execution.{ProjectExecTransformer, WholeStageTransformerSuite} -import org.apache.gluten.extension.GlutenPlan +import org.apache.gluten.execution.{ProjectExecTransformer, TransformSupport, WholeStageTransformerSuite} import org.apache.gluten.utils.{BackendTestUtils, SystemParameters} import org.apache.spark.SparkConf @@ -100,7 +99,8 @@ class GlutenExpressionDataTypesValidation extends WholeStageTransformerSuite { case _ => throw new UnsupportedOperationException("Not supported type: " + t) } } - def generateGlutenProjectPlan(expr: Expression): GlutenPlan = { + + def generateGlutenProjectPlan(expr: Expression): TransformSupport = { val namedExpr = Seq(Alias(expr, "r")()) ProjectExecTransformer(namedExpr, DummyPlan()) } diff --git a/pom.xml b/pom.xml index ee97ad5f90d3..3262fd503b38 100644 --- a/pom.xml +++ b/pom.xml @@ -65,8 +65,8 @@ delta-core 2.4.0 24 - 0.5.1 - 0.9.0 + 0.5.2 + 0.9.1 15.0.0 15.0.0-gluten arrow-memory-unsafe @@ -336,11 +336,11 @@ 3.5 spark-sql-columnar-shims-spark35 - 3.5.3 + 3.5.2 1.5.0 delta-spark - 3.2.1 - 32 + 3.2.0 + 32 0.15.0 2.15.1 3.3.4 @@ -530,11 +530,11 @@ - Darwin-x86 + darwin-x86 mac - x86 + x86_64 @@ -543,7 +543,7 @@ - Darwin-aarch64 + darwin-aarch64 mac @@ -556,7 +556,7 @@ - Linux-amd64 + linux-amd64 Linux @@ -569,7 +569,7 @@ - Linux-aarch64 + linux-aarch64 Linux diff --git a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala index a28a7d26b386..4f243f03fb78 100644 --- a/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala +++ b/shims/common/src/main/scala/org/apache/gluten/GlutenConfig.scala @@ -17,7 +17,7 @@ package org.apache.gluten import org.apache.spark.internal.Logging -import org.apache.spark.network.util.ByteUnit +import org.apache.spark.network.util.{ByteUnit, JavaUtils} import org.apache.spark.sql.internal.SQLConf import com.google.common.collect.ImmutableList @@ -50,12 +50,17 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableColumnarHiveTableScan: Boolean = conf.getConf(COLUMNAR_HIVETABLESCAN_ENABLED) + def enableColumnarHiveTableScanNestedColumnPruning: Boolean = + conf.getConf(COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED) + def enableVanillaVectorizedReaders: Boolean = conf.getConf(VANILLA_VECTORIZED_READERS_ENABLED) def enableColumnarHashAgg: Boolean = conf.getConf(COLUMNAR_HASHAGG_ENABLED) def forceToUseHashAgg: Boolean = conf.getConf(COLUMNAR_FORCE_HASHAGG_ENABLED) + def mergeTwoPhasesAggEnabled: Boolean = conf.getConf(MERGE_TWO_PHASES_ENABLED) + def enableColumnarProject: Boolean = conf.getConf(COLUMNAR_PROJECT_ENABLED) def enableColumnarFilter: Boolean = conf.getConf(COLUMNAR_FILTER_ENABLED) @@ -75,12 +80,14 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableNativeColumnarToRow: Boolean = conf.getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) - def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED) + def forceShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED) def enableColumnarSortMergeJoin: Boolean = conf.getConf(COLUMNAR_SORTMERGEJOIN_ENABLED) def enableColumnarUnion: Boolean = conf.getConf(COLUMNAR_UNION_ENABLED) + def enableNativeUnion: Boolean = conf.getConf(NATIVE_UNION_ENABLED) + def enableColumnarExpand: Boolean = conf.getConf(COLUMNAR_EXPAND_ENABLED) def enableColumnarBroadcastExchange: Boolean = conf.getConf(COLUMNAR_BROADCAST_EXCHANGE_ENABLED) @@ -107,6 +114,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCountDistinctWithoutExpand: Boolean = conf.getConf(ENABLE_COUNT_DISTINCT_WITHOUT_EXPAND) + def enableExtendedColumnPruning: Boolean = + conf.getConf(ENABLE_EXTENDED_COLUMN_PRUNING) + def veloxOrcScanEnabled: Boolean = conf.getConf(VELOX_ORC_SCAN_ENABLED) @@ -190,6 +200,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def columnarShuffleCompressionThreshold: Int = conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) + def columnarShuffleReaderBufferSize: Long = + conf.getConf(COLUMNAR_SHUFFLE_READER_BUFFER_SIZE) + def maxBatchSize: Int = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) def columnarToRowMemThreshold: Long = @@ -291,7 +304,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { def veloxSsdODirectEnabled: Boolean = conf.getConf(COLUMNAR_VELOX_SSD_ODIRECT_ENABLED) def veloxConnectorIOThreads: Int = { - conf.getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS) + conf.getConf(COLUMNAR_VELOX_CONNECTOR_IO_THREADS).getOrElse(numTaskSlotsPerExecutor) } def veloxSplitPreloadPerDriver: Integer = conf.getConf(COLUMNAR_VELOX_SPLIT_PRELOAD_PER_DRIVER) @@ -412,9 +425,10 @@ class GlutenConfig(conf: SQLConf) extends Logging { def debug: Boolean = conf.getConf(DEBUG_ENABLED) def debugKeepJniWorkspace: Boolean = conf.getConf(DEBUG_KEEP_JNI_WORKSPACE) def collectUtStats: Boolean = conf.getConf(UT_STATISTIC) - def taskStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) - def taskPartitionId: Int = conf.getConf(BENCHMARK_TASK_PARTITIONID) - def taskId: Long = conf.getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) + def benchmarkPartitionId: String = conf.getConf(BENCHMARK_TASK_PARTITIONID) + def benchmarkTaskId: String = conf.getConf(BENCHMARK_TASK_TASK_ID) + def benchmarkSaveDir: String = conf.getConf(BENCHMARK_SAVE_DIR) def textInputMaxBlockSize: Long = conf.getConf(TEXT_INPUT_ROW_MAX_BLOCK_SIZE) def textIputEmptyAsDefault: Boolean = conf.getConf(TEXT_INPUT_EMPTY_AS_DEFAULT) def enableParquetRowGroupMaxMinIndex: Boolean = @@ -444,7 +458,7 @@ class GlutenConfig(conf: SQLConf) extends Logging { conf.getConf(PREFETCH_ROW_GROUPS) def loadQuantum: Long = conf.getConf(LOAD_QUANTUM) - def maxCoalescedDistanceBytes: Long = + def maxCoalescedDistance: String = conf.getConf(MAX_COALESCED_DISTANCE_BYTES) def maxCoalescedBytes: Long = conf.getConf(MAX_COALESCED_BYTES) @@ -463,14 +477,14 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableCastAvgAggregateFunction: Boolean = conf.getConf(COLUMNAR_NATIVE_CAST_AGGREGATE_ENABLED) - def enableGlutenCostEvaluator: Boolean = conf.getConf(COST_EVALUATOR_ENABLED) - def dynamicOffHeapSizingEnabled: Boolean = conf.getConf(DYNAMIC_OFFHEAP_SIZING_ENABLED) def enableHiveFileFormatWriter: Boolean = conf.getConf(NATIVE_HIVEFILEFORMAT_WRITER_ENABLED) def enableCelebornFallback: Boolean = conf.getConf(CELEBORN_FALLBACK_ENABLED) + + def enableHdfsViewfs: Boolean = conf.getConf(HDFS_VIEWFS_ENABLED) } object GlutenConfig { @@ -556,6 +570,7 @@ object GlutenConfig { val SPARK_OFFHEAP_SIZE_KEY = "spark.memory.offHeap.size" val SPARK_OFFHEAP_ENABLED = "spark.memory.offHeap.enabled" val SPARK_REDACTION_REGEX = "spark.redaction.regex" + val SPARK_SHUFFLE_FILE_BUFFER = "spark.shuffle.file.buffer" // For Soft Affinity Scheduling // Enable Soft Affinity Scheduling, default value is false @@ -606,6 +621,9 @@ object GlutenConfig { val GLUTEN_SHUFFLE_WRITER_MERGE_THRESHOLD = "spark.gluten.sql.columnar.shuffle.merge.threshold" val GLUTEN_SHUFFLE_DEFUALT_COMPRESSION_BUFFER_SIZE = 32 * 1024 + // Shuffle reader buffer size. + val GLUTEN_SHUFFLE_READER_BUFFER_SIZE = "spark.gluten.sql.columnar.shuffle.readerBufferSize" + // Controls whether to load DLL from jars. User can get dependent native libs packed into a jar // by executing dev/package.sh. Then, with that jar configured, Gluten can load the native libs // at runtime. This config is just for velox backend. And it is NOT applicable to the situation @@ -645,6 +663,7 @@ object GlutenConfig { "spark.gluten.memory.dynamic.offHeap.sizing.memory.fraction" val GLUTEN_COST_EVALUATOR_ENABLED = "spark.gluten.sql.adaptive.costEvaluator.enabled" + val GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE = true var ins: GlutenConfig = _ @@ -720,6 +739,15 @@ object GlutenConfig { ) keyWithDefault.forEach(e => nativeConfMap.put(e._1, conf.getOrElse(e._1, e._2))) + conf + .get(SPARK_SHUFFLE_FILE_BUFFER) + .foreach( + v => + nativeConfMap + .put( + SPARK_SHUFFLE_FILE_BUFFER, + (JavaUtils.byteStringAs(v, ByteUnit.KiB) * 1024).toString)) + // Backend's dynamic session conf only. val confPrefix = prefixOf(backendName) conf @@ -759,7 +787,9 @@ object GlutenConfig { (AWS_S3_RETRY_MODE.key, AWS_S3_RETRY_MODE.defaultValueString), ( COLUMNAR_VELOX_CONNECTOR_IO_THREADS.key, - COLUMNAR_VELOX_CONNECTOR_IO_THREADS.defaultValueString), + conf.getOrElse( + NUM_TASK_SLOTS_PER_EXECUTOR.key, + NUM_TASK_SLOTS_PER_EXECUTOR.defaultValueString)), (COLUMNAR_SHUFFLE_CODEC.key, ""), (COLUMNAR_SHUFFLE_CODEC_BACKEND.key, ""), ("spark.hadoop.input.connect.timeout", "180000"), @@ -787,7 +817,8 @@ object GlutenConfig { SPARK_OFFHEAP_ENABLED, SESSION_LOCAL_TIMEZONE.key, DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key, - SPARK_REDACTION_REGEX + SPARK_REDACTION_REGEX, + LEGACY_TIME_PARSER_POLICY.key ) nativeConfMap.putAll(conf.filter(e => keys.contains(e._1)).asJava) @@ -855,6 +886,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val COLUMNAR_HIVETABLESCAN_NESTED_COLUMN_PRUNING_ENABLED = + buildConf("spark.gluten.sql.columnar.enableNestedColumnPruningInHiveTableScan") + .internal() + .doc("Enable or disable nested column pruning in hivetablescan.") + .booleanConf + .createWithDefault(true) + val VANILLA_VECTORIZED_READERS_ENABLED = buildStaticConf("spark.gluten.sql.columnar.enableVanillaVectorizedReaders") .internal() @@ -876,6 +914,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val MERGE_TWO_PHASES_ENABLED = + buildConf("spark.gluten.sql.mergeTwoPhasesAggregate.enabled") + .internal() + .doc("Whether to merge two phases aggregate if there are no other operators between them.") + .booleanConf + .createWithDefault(true) + val COLUMNAR_PROJECT_ENABLED = buildConf("spark.gluten.sql.columnar.project") .internal() @@ -937,7 +982,7 @@ object GlutenConfig { .booleanConf .createWithDefault(true) - val COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED = + val COLUMNAR_FORCE_SHUFFLED_HASH_JOIN_ENABLED = buildConf("spark.gluten.sql.columnar.forceShuffledHashJoin") .internal() .booleanConf @@ -979,6 +1024,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val NATIVE_UNION_ENABLED = + buildConf("spark.gluten.sql.native.union") + .internal() + .doc("Enable or disable native union where computation is completely offloaded to backend.") + .booleanConf + .createWithDefault(false) + val COLUMNAR_EXPAND_ENABLED = buildConf("spark.gluten.sql.columnar.expand") .internal() @@ -1151,6 +1203,13 @@ object GlutenConfig { .checkValue(v => v >= 0 && v <= 1, "Shuffle writer merge threshold must between [0, 1]") .createWithDefault(0.25) + val COLUMNAR_SHUFFLE_READER_BUFFER_SIZE = + buildConf(GLUTEN_SHUFFLE_READER_BUFFER_SIZE) + .internal() + .doc("Buffer size in bytes for shuffle reader reading input stream from local or remote.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("1MB") + val COLUMNAR_MAX_BATCH_SIZE = buildConf(GLUTEN_MAX_BATCH_SIZE_KEY) .internal() @@ -1357,9 +1416,10 @@ object GlutenConfig { val RAS_ENABLED = buildConf("spark.gluten.ras.enabled") .doc( - "Experimental: Enables RAS (relational algebra selector) during physical " + - "planning to generate more efficient query plan. Note, this feature is still in " + - "development and may not bring performance profits.") + "Enables RAS (relational algebra selector) during physical " + + "planning to generate more efficient query plan. Note, this feature doesn't bring " + + "performance profits by default. Try exploring option `spark.gluten.ras.costModel` " + + "for advanced usage.") .booleanConf .createWithDefault(false) @@ -1414,6 +1474,13 @@ object GlutenConfig { .bytesConf(ByteUnit.BYTE) .createWithDefaultString("8MB") + val COLUMNAR_VELOX_MEM_RECLAIM_MAX_WAIT_MS = + buildConf("spark.gluten.sql.columnar.backend.velox.reclaimMaxWaitMs") + .internal() + .doc("The max time in ms to wait for memory reclaim.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(TimeUnit.MINUTES.toMillis(60)) + val COLUMNAR_VELOX_SSD_CACHE_PATH = buildStaticConf("spark.gluten.sql.columnar.backend.velox.ssdCachePath") .internal() @@ -1449,19 +1516,15 @@ object GlutenConfig { .booleanConf .createWithDefault(false) - // FIXME: May cause issues when toggled on. Examples: - // https://github.com/apache/incubator-gluten/issues/7161 - // https://github.com/facebookincubator/velox/issues/10173 val COLUMNAR_VELOX_CONNECTOR_IO_THREADS = buildStaticConf("spark.gluten.sql.columnar.backend.velox.IOThreads") .internal() .doc( - "Experimental: The Size of the IO thread pool in the Connector." + - " This thread pool is used for split preloading and DirectBufferedInput." + - " The option is experimental. Toggling on it (setting a non-zero value) may cause some" + - " unexpected issues when application reaches some certain conditions.") + "The Size of the IO thread pool in the Connector. " + + "This thread pool is used for split preloading and DirectBufferedInput. " + + "By default, the value is the same as the maximum task slots per Spark executor.") .intConf - .createWithDefault(0) + .createOptional val COLUMNAR_VELOX_ASYNC_TIMEOUT = buildStaticConf("spark.gluten.sql.columnar.backend.velox.asyncTimeoutOnTaskStopping") @@ -1678,14 +1741,20 @@ object GlutenConfig { val BENCHMARK_TASK_PARTITIONID = buildConf("spark.gluten.sql.benchmark_task.partitionId") .internal() - .intConf - .createWithDefault(-1) + .stringConf + .createWithDefault("") val BENCHMARK_TASK_TASK_ID = buildConf("spark.gluten.sql.benchmark_task.taskId") .internal() - .longConf - .createWithDefault(-1L) + .stringConf + .createWithDefault("") + + val BENCHMARK_SAVE_DIR = + buildConf(GLUTEN_SAVE_DIR) + .internal() + .stringConf + .createWithDefault("") val NATIVE_WRITER_ENABLED = buildConf("spark.gluten.sql.native.writer.enabled") @@ -1931,6 +2000,13 @@ object GlutenConfig { .booleanConf .createWithDefault(false) + val ENABLE_EXTENDED_COLUMN_PRUNING = + buildConf("spark.gluten.sql.extendedColumnPruning.enabled") + .internal() + .doc("Do extended nested column pruning for cases ignored by vanilla Spark.") + .booleanConf + .createWithDefault(true) + val COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS = buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.expectedNumItems") .internal() @@ -2029,11 +2105,11 @@ object GlutenConfig { .createWithDefaultString("256MB") val MAX_COALESCED_DISTANCE_BYTES = - buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedDistanceBytes") + buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedDistance") .internal() .doc(" Set the max coalesced distance bytes for velox file scan") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1MB") + .stringConf + .createWithDefaultString("512KB") val MAX_COALESCED_BYTES = buildStaticConf("spark.gluten.sql.columnar.backend.velox.maxCoalescedBytes") @@ -2108,15 +2184,15 @@ object GlutenConfig { .createWithDefault(true) val COST_EVALUATOR_ENABLED = - buildConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) + buildStaticConf(GlutenConfig.GLUTEN_COST_EVALUATOR_ENABLED) .internal() .doc( - "If true and gluten enabled, use " + + "If true, use " + "org.apache.spark.sql.execution.adaptive.GlutenCostEvaluator as custom cost " + "evaluator class, else follow the configuration " + "spark.sql.adaptive.customCostEvaluatorClass.") .booleanConf - .createWithDefault(true) + .createWithDefault(GLUTEN_COST_EVALUATOR_ENABLED_DEFAULT_VALUE) val DYNAMIC_OFFHEAP_SIZING_ENABLED = buildConf(GlutenConfig.GLUTEN_DYNAMIC_OFFHEAP_SIZING_ENABLED) @@ -2163,4 +2239,11 @@ object GlutenConfig { "Otherwise, do nothing.") .booleanConf .createWithDefault(false) + + val HDFS_VIEWFS_ENABLED = + buildStaticConf("spark.gluten.storage.hdfsViewfs.enabled") + .internal() + .doc("If enabled, gluten will convert the viewfs path to hdfs path in scala side") + .booleanConf + .createWithDefault(false) } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 2bdde3b4aaa0..a3bd5079b016 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -71,10 +71,9 @@ abstract class AbstractFileSourceScanExec( disableBucketedScan: Boolean = false) extends DataSourceScanExec { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 46b59ac306c2..f38c85a49dde 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -232,4 +233,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneDataSchema(schema, requestedFields) + } } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index c21c67f65481..821d8317d87d 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -43,6 +43,7 @@ import org.apache.hadoop.mapred.{JobConf, Reporter} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import scala.collection.JavaConverters._ + /* - * This class is copied from Spark 3.2 and modified for Gluten. \n * Gluten should make sure this class is loaded before the original class. @@ -101,19 +102,22 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") + val tableOptions = tableDesc.getProperties.asScala.toMap val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec } else if (isParquetFormat) { - val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + val parquetOptions = + new ParquetOptions(tableOptions, sparkSession.sessionState.conf) parquetOptions.compressionCodecClassName } else { - val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + val orcOptions = new OrcOptions(tableOptions, sparkSession.sessionState.conf) orcOptions.compressionCodec } - val nativeConf = GlutenFormatFactory(nativeFormat).nativeConf(options, compressionCodec) + val nativeConf = + GlutenFormatFactory(nativeFormat).nativeConf(tableOptions, compressionCodec) new OutputWriterFactory { private val jobConf = new SerializableJobConf(new JobConf(conf)) diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 6b3d679fcdd4..c885f0cf44b3 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -75,10 +75,9 @@ abstract class AbstractFileSourceScanExec( lazy val metadataColumns: Seq[AttributeReference] = output.collect { case FileSourceMetadataAttribute(attr) => attr } - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index dd095f0ff247..d9b6bb936f67 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -239,4 +240,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 6ed1b4d21536..61ad8fe72bc8 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -98,19 +98,22 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName if ("true" == sparkSession.sparkContext.getLocalProperty("isNativeApplicable")) { val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") + val tableOptions = tableDesc.getProperties.asScala.toMap val isParquetFormat = nativeFormat == "parquet" val compressionCodec = if (fileSinkConf.compressed) { // hive related configurations fileSinkConf.compressCodec } else if (isParquetFormat) { - val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + val parquetOptions = + new ParquetOptions(tableOptions, sparkSession.sessionState.conf) parquetOptions.compressionCodecClassName } else { - val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + val orcOptions = new OrcOptions(tableOptions, sparkSession.sessionState.conf) orcOptions.compressionCodec } - val nativeConf = GlutenFormatFactory(nativeFormat).nativeConf(options, compressionCodec) + val nativeConf = + GlutenFormatFactory(nativeFormat).nativeConf(tableOptions, compressionCodec) new OutputWriterFactory { private val jobConf = new SerializableJobConf(new JobConf(conf)) diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 5e751861861a..53ea6f543a95 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala index cb25da3d3d20..3313c3c76842 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{KeyGroupedPartitioning, Partitioning, SinglePartition} -import org.apache.spark.sql.catalyst.util.{truncatedString, InternalRowComparableWrapper} +import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.read._ import org.apache.spark.sql.internal.SQLConf @@ -252,14 +252,7 @@ abstract class AbstractBatchScanExec( rdd } - override def simpleString(maxFields: Int): String = { - val truncatedOutputString = truncatedString(output, "[", ", ", "]", maxFields) - val runtimeFiltersString = s"RuntimeFilters: ${runtimeFilters.mkString("[", ",", "]")}" - val result = s"$nodeName$truncatedOutputString ${scan.description()} $runtimeFiltersString" - redact(result) - } - override def nodeName: String = { - s"BatchScan ${table.name()}".trim + s"BatchScanTransformer ${table.name()}".trim } } diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala index ad99f7be2547..eab32ab9d0b9 100644 --- a/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala +++ b/shims/spark35/src/main/scala/org/apache/gluten/sql/shims/spark35/SparkShimProvider.scala @@ -20,7 +20,7 @@ import org.apache.gluten.sql.shims.{SparkShimDescriptor, SparkShims} import org.apache.gluten.sql.shims.spark35.SparkShimProvider.DESCRIPTOR object SparkShimProvider { - val DESCRIPTOR = SparkShimDescriptor(3, 5, 3) + val DESCRIPTOR = SparkShimDescriptor(3, 5, 2) } class SparkShimProvider extends org.apache.gluten.sql.shims.SparkShimProvider { diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala index 32cbd435b013..c8dbcc2fed4f 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/AbstractFileSourceScanExec.scala @@ -67,15 +67,9 @@ abstract class AbstractFileSourceScanExec( override val disableBucketedScan: Boolean = false) extends FileSourceScanLike { - // Note that some vals referring the file-based relation are lazy intentionally - // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsColumnar: Boolean = { - val conf = relation.sparkSession.sessionState.conf - // Only output columnar if there is WSCG to read it. - val requiredWholeStageCodegenSettings = - conf.wholeStageEnabled && !WholeStageCodegenExec.isTooManyFields(conf, schema) - requiredWholeStageCodegenSettings && - relation.fileFormat.supportBatch(relation.sparkSession, schema) + override def supportsColumnar: Boolean = { + // The value should be defined in GlutenPlan. + throw new UnsupportedOperationException("Unreachable code") } private lazy val needsUnsafeRowConversion: Boolean = { diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala index 3aa3be6ead3d..8f51ea2c72b1 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AbstractBatchScanExec.scala @@ -260,6 +260,6 @@ abstract class AbstractBatchScanExec( } override def nodeName: String = { - s"BatchScan ${table.name()}".trim + s"BatchScanTransformer ${table.name()}".trim } } diff --git a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala index 87aba00b0f59..3521d496546d 100644 --- a/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala +++ b/shims/spark35/src/main/scala/org/apache/spark/sql/hive/execution/AbstractHiveTableScanExec.scala @@ -22,12 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.SchemaPruning.RootField import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{BooleanType, DataType} +import org.apache.spark.sql.types.{BooleanType, DataType, StructType} import org.apache.spark.util.Utils import org.apache.hadoop.conf.Configuration @@ -257,4 +258,8 @@ abstract private[hive] class AbstractHiveTableScanExec( } override def otherCopyArgs: Seq[AnyRef] = Seq(sparkSession) + + def pruneSchema(schema: StructType, requestedFields: Seq[RootField]): StructType = { + SchemaPruning.pruneSchema(schema, requestedFields) + } } diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index 24c277b7b2f1..c3f0a2a8f37e 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -22,7 +22,7 @@ 2.12 3 0.3.2-incubating - 0.9.0 + 0.9.1 1.3.0-SNAPSHOT 32.0.1-jre 1.1 @@ -164,7 +164,7 @@ spark-3.5 - 3.5.3 + 3.5.2 2.12.18 @@ -177,7 +177,7 @@ celeborn-0.5 - 0.5.1 + 0.5.2 diff --git a/tools/gluten-te/ubuntu/dockerfile-buildenv b/tools/gluten-te/ubuntu/dockerfile-buildenv index 47c7b80d8619..df13e8726190 100644 --- a/tools/gluten-te/ubuntu/dockerfile-buildenv +++ b/tools/gluten-te/ubuntu/dockerfile-buildenv @@ -108,9 +108,18 @@ RUN cd /opt && wget https://github.com/Kitware/CMake/releases/download/v3.28.3/c && mkdir cmake \ && bash cmake-3.28.3-linux-x86_64.sh --skip-license --prefix=/opt/cmake \ && ln -s /opt/cmake/bin/cmake /usr/bin/cmake - RUN cmake --version +# Install GCC 11 +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y software-properties-common +RUN add-apt-repository ppa:ubuntu-toolchain-r/test +RUN apt-get update && DEBIAN_FRONTEND=noninteractive apt-get install -y gcc-11 g++-11 +RUN rm -f /usr/bin/gcc /usr/bin/g++ +RUN ln -s /usr/bin/gcc-11 /usr/bin/gcc +RUN ln -s /usr/bin/g++-11 /usr/bin/g++ +RUN cc --version +RUN c++ --version + # Spark binaries WORKDIR /opt ARG BUILD_SPARK_BINARIES diff --git a/tools/workload/benchmark_velox/README.md b/tools/workload/benchmark_velox/README.md new file mode 100644 index 000000000000..5f080077f606 --- /dev/null +++ b/tools/workload/benchmark_velox/README.md @@ -0,0 +1,38 @@ +# Setup, Build and Benchmark Spark/Gluten with Jupyter Notebook + +This guide provides notebooks and scripts for conducting performance testing in Gluten. The standard approach involves setting up the test environment on a bare-metal machine or cloud instance and running performance tests with TPC-H/TPC-DS workloads. These scripts enable users to reproduce our performance results in their own environment. + +## Environment Setup + +The recommended OS is ubuntu22.04 with kernel 5.15. To prepare the environment, run [initialize.ipynb](./initialize.ipynb), which will: + +- Install system dependencies and set up jupyter notebook +- Configure Hadoop and Spark +- Configure kernel parameters +- Build Gluten using Docker +- Generate TPC-H/TPC-DS tables + +## Running TPC-H/TPC-DS Benchmarks + +To run TPC-H/TPC-DS benchmarks, use [tpc_workload.ipynb](./tpc_workload.ipynb). You can create a copy of the notebook and modify the parameters defined in this notebook to run different workloads. However, creating and modifying a copy each time you change workloads can be inconvenient. Instead, it's recommended to use Papermill to pass parameters via the command line for greater flexibility. + +The required parameters are specified in [params.yaml.template](./params.yaml.template). To use it, create your own YAML file by copying and modifying the template. The command to run the notebook is: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml gluten_tpch.ipynb +``` +After execution, the output notebook will be saved as `gluten_tpch.ipynb`. + +If you want to use different parameters, you can specify them via the `-p` option. It will overwrite the previously defined parameters in `params.yaml`. e.g. To switch to the TPC-DS workload, run: + +```bash +papermill tpc_workload.ipynb --inject-output-path -f params.yaml -p workoad tpcds gluten_tpcds.ipynb +``` + +Please refer to the Papermill documentation for additional usage details. + +We also provide a script [run_tpc_workload.sh](./run_tpc_workload.sh). This script wraps the Papermill command, automatically renaming the output notebook with a timestamp and application ID to prevent overwriting existing output files. + +## Analyzing Performance Results + +You can check the **Show Performance** section in the output notebook after execution. It shows the cpu% per query, and draws charts for the cpu%, memory throughput, disk throughput/util%, network throughput and pagefaults. diff --git a/tools/workload/benchmark_velox/init_disks.py b/tools/workload/benchmark_velox/init_disks.py new file mode 100644 index 000000000000..8e47e16e3078 --- /dev/null +++ b/tools/workload/benchmark_velox/init_disks.py @@ -0,0 +1,103 @@ +# 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. + +# To set up the virtual environment required to run this script, +# refer to the `Format and mount disks` subsection under `System Setup` in initialize.ipynb. +import sys +import subprocess +import questionary +import json + +def yes_or_no(question): + while True: + user_input = input(question + '(yes/no/quit): ') + if user_input.lower() == 'yes': + return True + elif user_input.lower() == 'no': + return False + elif user_input.lower() == 'quit': + sys.exit(1) + else: + continue + +def filter_empty_str(l): + return [x for x in l if x] + +def run_and_log(cmd): + # Print command in yellow + print('\033[93m' + '>>> Running command: ' + repr(cmd) + '\033[0m') + result = subprocess.run(cmd, check=True, shell=True, capture_output=True, text=True) + # Print stdout in green + print('\033[92m' + '==========stdout==========' + '\033[0m') + print(result.stdout) + # Print stderr in red + print('\033[91m' + '==========stderr==========' + '\033[0m') + print(result.stderr) + +def init_disks(): + all_disks = filter_empty_str(subprocess.run("lsblk -I 7,8,259 -npd --output NAME".split(' '), capture_output=True, text=True).stdout.split('\n')) + if not all_disks: + print("No disks found on system. Exit.") + sys.exit(0) + + answer = False + disks = [] + while not answer: + disks = questionary.checkbox('Select disks to initialize:', choices=all_disks).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(disks) + '\n') + + if not disks: + print('No disks are selected.') + return + + for d in disks: + print('Initializing {} ...'.format(d)) + run_and_log('wipefs -a {}'.format(d)) + run_and_log('echo "g\nw\n" | fdisk {}'.format(d)) + run_and_log('echo "n\n\n\n\nw\n" | fdisk {}'.format(d)) + run_and_log('mkfs.ext4 {}p1'.format(d)) + +def mount_partitions(): + subprocess.run('lsblk -pf --json > lsblk.json', shell=True) + partitions = [] + with open('lsblk.json', 'r') as f: + data = json.load(f) + for d in data['blockdevices']: + if 'children' in d: + for c in d['children']: + if c['fstype'] == 'ext4': + partitions.append(c['name']) + answer = False + while not answer: + partitions = questionary.checkbox('Select partitions to create mount points:', choices=partitions).ask() + answer = yes_or_no('Confirm selected:\n' + '\n'.join(partitions) + '\n') + + for i, p in enumerate(partitions): + d = 'data{}'.format(i) + run_and_log('e2label {} ""'.format(p)) + run_and_log('e2label {} {}'.format(p, d)) + run_and_log('mkdir -p /{}'.format(d)) + run_and_log('mount -L {} /{}'.format(d, d)) + +def choose(): + choice = questionary.select('Select operation:', choices=['Format disks', 'Mount partitions']).ask() + print(choice) + if choice == 'Format disks': + init_disks() + elif choice == 'Mount partitions': + mount_partitions() + +if __name__ == '__main__': + choose() diff --git a/tools/workload/benchmark_velox/initialize.ipynb b/tools/workload/benchmark_velox/initialize.ipynb new file mode 100644 index 000000000000..cbbc27686951 --- /dev/null +++ b/tools/workload/benchmark_velox/initialize.ipynb @@ -0,0 +1,2918 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# System Setup" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**1. Install system dependencies and python packages. Prepare the environment.**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "First, install all dependencies and python packages as `root`. Run commands and make sure the installations are successful.\n", + "\n", + "```bash\n", + "apt update\n", + "\n", + "apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex\n", + "\n", + "python3 -m pip install notebook==6.5.2\n", + "python3 -m pip install jupyter_server==1.23.4\n", + "python3 -m pip install jupyter_highlight_selected_word\n", + "python3 -m pip install jupyter_contrib_nbextensions\n", + "python3 -m pip install virtualenv==20.21.1\n", + "python3 -m pip uninstall -y ipython\n", + "python3 -m pip install ipython==8.21.0\n", + "python3 -m pip uninstall -y traitlets\n", + "python3 -m pip install traitlets==5.9.0\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Check that there isn't an entry for your hostname mapped to 127.0.0.1 or 127.0.1.1 in /etc/hosts (Ubuntu is notorious for this). If there is, delete it.\n", + "Then add `` and `` for master and worker nodes.\n", + "\n", + "Example /etc/hosts:\n", + " \n", + "```\n", + "127.0.0.1 localhost\n", + "\n", + "# The following lines are desirable for IPv6 capable hosts\n", + "::1 ip6-localhost ip6-loopback\n", + "fe00::0 ip6-localnet\n", + "ff00::0 ip6-mcastprefix\n", + "ff02::1 ip6-allnodes\n", + "ff02::2 ip6-allrouters\n", + "\n", + "10.0.0.117 sr217\n", + "10.0.0.113 sr213\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**2. Format and mount disks**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create a python virtual environment to finish the system setup process:\n", + "\n", + "```bash\n", + "virtualenv -p python3 -v venv\n", + "source venv/bin/activate\n", + "```\n", + "\n", + "And install packages under `venv`:\n", + "```bash\n", + "(venv) python3 -m pip install questionary\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Run script [init_disks.py](./init_disks.py) to format and mount disks. **Be careful when choosing the disks to format.** If you see errors like `device or resource busy`, perhaps the partition has been mounted, you should unmount it first. If you still see this error, reboot the system and try again." + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Exit `venv`:\n", + "```bash\n", + "(venv) deactivate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**3. Create user `sparkuser`**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Create user `sparkuser` without password and with sudo priviledge. It's recommended to use one of the disks as the home directory instead of the system drive.\n", + "\n", + "```bash\n", + "mkdir -p /data0/home/sparkuser\n", + "ln -s /data0/home/sparkuser /home/sparkuser\n", + "cp -r /etc/skel/. /home/sparkuser/\n", + "adduser --home /home/sparkuser --disabled-password --gecos \"\" sparkuser\n", + "\n", + "chown -R sparkuser:sparkuser /data*\n", + "\n", + "echo 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `sparkuser`\n", + "\n", + "```bashrc\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bashrc\n", + "rm -rf ~/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat ~/.ssh/id_rsa.pub >> ~/.ssh/authorized_keys\n", + "\n", + "exit\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Generate ssh keys for `root`, and enable no password ssh from `sparkuser`\n", + "\n", + "```bash\n", + "rm -rf /root/.ssh\n", + "ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa <</dev/null 2>&1\n", + "cat /root/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "cat /home/sparkuser/.ssh/id_rsa.pub >> /root/.ssh/authorized_keys\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Login to `sparkuser` and run the first-time ssh to the `root`\n", + "\n", + "```bash\n", + "su - sparkuser\n", + "```\n", + "\n", + "```bash\n", + "ssh -o StrictHostKeyChecking=no root@localhost ls\n", + "ssh -o StrictHostKeyChecking=no root@127.0.0.1 ls\n", + "ssh -o StrictHostKeyChecking=no root@`hostname` ls\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from `If not running interactively, don't do anything` in ~/.bashrc\n", + "\n", + "```bash\n", + "sed -i '5,9 s/^/# /' ~/.bashrc\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "**4. Configure jupyter notebook**" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "As `sparkuser`, install python packages\n", + "\n", + "```bash\n", + "cd /home/sparkuser/.local/lib/ && rm -rf python*\n", + "\n", + "python3 -m pip install --upgrade jsonschema\n", + "python3 -m pip install jsonschema[format]\n", + "python3 -m pip install sqlalchemy==1.4.46\n", + "python3 -m pip install papermill Black\n", + "python3 -m pip install NotebookScripter\n", + "python3 -m pip install findspark spylon-kernel matplotlib pandasql pyhdfs\n", + "python3 -m pip install ipywidgets jupyter_nbextensions_configurator ipyparallel\n", + "```\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Configure jupyter notebook. Setup password when it prompts\n", + "\n", + "```bash\n", + "jupyter notebook --generate-config\n", + "\n", + "jupyter notebook password\n", + "\n", + "mkdir -p ~/.jupyter/custom/\n", + "\n", + "echo '.container { width:100% !important; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "echo 'div.output_stderr { background: #ffdd; display: none; }' >> ~/.jupyter/custom/custom.css\n", + "\n", + "jupyter nbextension install --py jupyter_highlight_selected_word --user\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension install --py widgetsnbextension --user\n", + "\n", + "jupyter contrib nbextension install --user\n", + "\n", + "jupyter nbextension enable codefolding/main\n", + "\n", + "jupyter nbextension enable code_prettify/code_prettify\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable code_font_size/code_font_size\n", + "\n", + "jupyter nbextension enable collapsible_headings/main\n", + "\n", + "jupyter nbextension enable highlight_selected_word/main\n", + "\n", + "jupyter nbextension enable ipyparallel/main\n", + "\n", + "jupyter nbextension enable move_selected_cells/main\n", + "\n", + "jupyter nbextension enable nbTranslate/main\n", + "\n", + "jupyter nbextension enable scratchpad/main\n", + "\n", + "jupyter nbextension enable tree-filter/index\n", + "\n", + "jupyter nbextension enable comment-uncomment/main\n", + "\n", + "jupyter nbextension enable export_embedded/main\n", + "\n", + "jupyter nbextension enable hide_header/main\n", + "\n", + "jupyter nbextension enable highlighter/highlighter\n", + "\n", + "jupyter nbextension enable scroll_down/main\n", + "\n", + "jupyter nbextension enable snippets/main\n", + "\n", + "jupyter nbextension enable toc2/main\n", + "\n", + "jupyter nbextension enable varInspector/main\n", + "\n", + "jupyter nbextension enable codefolding/edit\n", + "\n", + "jupyter nbextension enable contrib_nbextensions_help_item/main\n", + "\n", + "jupyter nbextension enable freeze/main\n", + "\n", + "jupyter nbextension enable hide_input/main\n", + "\n", + "jupyter nbextension enable jupyter-js-widgets/extension\n", + "\n", + "jupyter nbextension enable snippets_menu/main\n", + "\n", + "jupyter nbextension enable table_beautifier/main\n", + "\n", + "jupyter nbextension enable hide_input_all/main\n", + "\n", + "jupyter nbextension enable spellchecker/main\n", + "\n", + "jupyter nbextension enable toggle_all_line_numbers/main\n", + "\n", + "jupyter nbextensions_configurator enable --user\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone Gluten\n", + "\n", + "```bash\n", + "cd ~\n", + "git clone https://github.com/apache/incubator-gluten.git gluten\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Start jupyter notebook\n", + "\n", + "```bash\n", + "mkdir -p ~/ipython\n", + "cd ~/ipython\n", + "\n", + "nohup jupyter notebook --ip=0.0.0.0 --port=8888 &\n", + "\n", + "cp ~/gluten/tools/workload/benchmark_velox ~/ipython/\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Initialize\n", + " Run this section after notebook restart! " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify datadir. The directories are used for spark.local.dirs and hadoop namenode/datanode." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "datadir=[f'/data{i}' for i in range(0, 8)]\n", + "datadir" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify clients(workers). Leave it empty if the cluster is setup on the local machine." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "clients=''''''.split()\n", + "print(clients)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Specify JAVA_HOME" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "java_home = '/usr/lib/jvm/java-8-openjdk-amd64'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "import platform\n", + "\n", + "user=os.getenv('USER')\n", + "print(f\"user: {user}\")\n", + "print()\n", + "\n", + "masterip=socket.gethostbyname(socket.gethostname())\n", + "hostname=socket.gethostname() \n", + "print(f\"masterip: {masterip} hostname: {hostname}\")\n", + "print()\n", + "\n", + "hclients=clients.copy()\n", + "hclients.append(hostname)\n", + "print(f\"master and workers: {hclients}\")\n", + "print()\n", + "\n", + "\n", + "if clients:\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"lscpu | grep '^CPU(s)'\\\"\" + \" | awk '{print $2}'\"\n", + " client_cpu = !{cmd}\n", + " cpu_num = client_cpu[0]\n", + "\n", + " cmd = f\"ssh {clients[0]} \" + \"\\\"cat /proc/meminfo | grep MemTotal\\\"\" + \" | awk '{print $2}'\"\n", + " totalmemory = !{cmd}\n", + " totalmemory = int(totalmemory[0])\n", + "else:\n", + " cpu_num = os.cpu_count()\n", + " totalmemory = !cat /proc/meminfo | grep MemTotal | awk '{print $2}'\n", + " totalmemory = int(totalmemory[0])\n", + " \n", + "print(f\"cpu_num: {cpu_num}\")\n", + "print()\n", + "\n", + "print(\"total memory: \", totalmemory, \"KB\")\n", + "print()\n", + "\n", + "mem_mib = int(totalmemory/1024)-1024\n", + "print(f\"mem_mib: {mem_mib}\")\n", + "print()\n", + "\n", + "is_arm = platform.machine() == 'aarch64'\n", + "print(\"is_arm: \",is_arm)\n", + "print()\n", + "\n", + "sparklocals=\",\".join([f'{l}/{user}/yarn/local' for l in datadir])\n", + "print(f\"SPARK_LOCAL_DIR={sparklocals}\")\n", + "print()\n", + "\n", + "%cd ~" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Set up clients\n", + " SKIP for single node " + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install dependencies" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually configure ssh login without password to all clients\n", + "\n", + "```bash\n", + "ssh-copy-id -o StrictHostKeyChecking=no root@\n", + "```" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y sudo locales wget tar tzdata git ccache cmake ninja-build build-essential llvm-11-dev clang-11 libiberty-dev libdwarf-dev libre2-dev libz-dev libssl-dev libboost-all-dev libcurl4-openssl-dev openjdk-8-jdk maven vim pip sysstat gcc-9 libjemalloc-dev nvme-cli curl zip unzip bison flex > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create user" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh -o StrictHostKeyChecking=no root@{l} ls" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} adduser --disabled-password --gecos '\"\"' {user}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} cp -r .ssh /home/{user}/\n", + " !ssh root@{l} chown -R {user}:{user} /home/{user}/.ssh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} \"echo -e 'sparkuser ALL=(ALL:ALL) NOPASSWD:ALL' | EDITOR='tee -a' visudo\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "***Required for Ubuntu***\n", + "\n", + "Run below command to comment out lines starting from If not running interactively, don't do anything in ~/.bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh {l} sed -i \"'5,9 s/^/# /'\" ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Use /etc/hosts on master node" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp /etc/hosts root@{l}:/etc/hosts" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Setup disks" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !ssh root@{l} apt update > /dev/null 2>&1\n", + " !ssh root@{l} apt install -y pip > /dev/null 2>&1\n", + " !ssh root@{l} python3 -m pip install virtualenv" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Manually run **2. Format and mount disks** section under [System Setup](#System-Setup)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure Spark, Hadoop" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Download packages" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/spark/spark-3.3.1/spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!wget https://archive.apache.org/dist/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "# backup url: !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.2.4/hadoop-3.2.4.tar.gz > /dev/null 2>&1\n", + "if is_arm:\n", + " # download both versions\n", + " !wget https://dlcdn.apache.org/hadoop/common/hadoop-3.3.5/hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Create directories" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"chown -R {user}:{user} \" + l for l in datadir])\n", + "for l in hclients:\n", + " !ssh root@{l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"rm -rf {l}/tmp; mkdir -p {l}/tmp\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "cmd=\";\".join([f\"mkdir -p {l}/{user}/hdfs/data; mkdir -p {l}/{user}/yarn/local\" for l in datadir])\n", + "for l in hclients:\n", + " !ssh {l} '{cmd}'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p {datadir[0]}/{user}/hdfs/name\n", + "!mkdir -p {datadir[0]}/{user}/hdfs/namesecondary" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp hadoop-3.2.4.tar.gz {l}:~/\n", + " !scp spark-3.3.1-bin-hadoop3.tgz {l}:~/\n", + " !ssh {l} \"mv -f hadoop hadoop.bak; mv -f spark spark.bak\"\n", + " !ssh {l} \"tar zxvf hadoop-3.2.4.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"tar -zxvf spark-3.3.1-bin-hadoop3.tgz > /dev/null 2>&1\"\n", + " !ssh root@{l} \"apt install -y openjdk-8-jdk > /dev/null 2>&1\"\n", + " !ssh {l} \"ln -s hadoop-3.2.4 hadoop; ln -s spark-3.3.1-bin-hadoop3 spark\"\n", + " if is_arm:\n", + " !ssh {l} \"tar zxvf hadoop-3.3.5-aarch64.tar.gz > /dev/null 2>&1\"\n", + " !ssh {l} \"cd hadoop && mv lib lib.bak && cp -rf ~/hadoop-3.3.5/lib ~/hadoop\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure bashrc" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "\n", + "cfg=f'''export HADOOP_HOME=~/hadoop\n", + "export PATH=$PATH:$HADOOP_HOME/bin:$HADOOP_HOME/sbin\n", + "\n", + "export HADOOP_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "export YARN_CONF_DIR=$HADOOP_HOME/etc/hadoop\n", + "\n", + "export SPARK_HOME=~/spark\n", + "export PYTHONPATH=$SPARK_HOME/python:$SPARK_HOME/python/lib/py4j-0.10.9.5-src.zip:$PYTHONPATH\n", + "export PATH=$SPARK_HOME/bin:$PATH\n", + "\n", + "'''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " cfg += 'export CPU_TARGET=\"aarch64\"\\nexport JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'\n", + "else:\n", + " cfg += f'export JAVA_HOME={java_home}\\nexport PATH=$JAVA_HOME/bin:$PATH\\n'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "with open(\"tmpcfg\",'w') as f:\n", + " f.writelines(cfg)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp tmpcfg {l}:~/tmpcfg.in\n", + " !ssh {l} \"cat ~/tmpcfg.in >> ~/.bashrc\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Hadoop" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"apt install -y libiberty-dev libxml2-dev libkrb5-dev libgsasl7-dev libuuid1 uuid-dev > /dev/null 2>&1\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### setup short-circuit " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh root@{l} \"mkdir -p /var/lib/hadoop-hdfs/\"\n", + " !ssh root@{l} 'chown {user}:{user} /var/lib/hadoop-hdfs/'" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### enable security.authorization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "coresite='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " fs.default.name\n", + " hdfs://{:s}:8020\n", + " true\n", + " \n", + " \n", + " hadoop.security.authentication\n", + " simple\n", + " \n", + " \n", + " hadoop.security.authorization\n", + " true\n", + " \n", + "\n", + "'''.format(hostname)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/core-site.xml','w') as f:\n", + " f.writelines(coresite)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/core-site.xml {l}:~/hadoop/etc/hadoop/core-site.xml >/dev/null 2>&1" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### set IP check, note the command \", \".join" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "hadooppolicy='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " security.service.authorization.default.hosts\n", + " {:s}\n", + " \n", + " \n", + " security.service.authorization.default.acl\n", + " {:s} {:s}\n", + " \n", + " \n", + " \n", + " \n", + " security.client.protocol.acl\n", + " *\n", + " ACL for ClientProtocol, which is used by user code\n", + " via the DistributedFileSystem.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.client.datanode.protocol.acl\n", + " *\n", + " ACL for ClientDatanodeProtocol, the client-to-datanode protocol\n", + " for block recovery.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.datanode.protocol.acl\n", + " *\n", + " ACL for DatanodeProtocol, which is used by datanodes to\n", + " communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.inter.datanode.protocol.acl\n", + " *\n", + " ACL for InterDatanodeProtocol, the inter-datanode protocol\n", + " for updating generation timestamp.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.namenode.protocol.acl\n", + " *\n", + " ACL for NamenodeProtocol, the protocol used by the secondary\n", + " namenode to communicate with the namenode.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.admin.operations.protocol.acl\n", + " *\n", + " ACL for AdminOperationsProtocol. Used for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.user.mappings.protocol.acl\n", + " *\n", + " ACL for RefreshUserMappingsProtocol. Used to refresh\n", + " users mappings. The ACL is a comma-separated list of user and\n", + " group names. The user and group list is separated by a blank. For\n", + " e.g. \"alice,bob users,wheel\". A special value of \"*\" means all\n", + " users are allowed.\n", + " \n", + "\n", + " \n", + " security.refresh.policy.protocol.acl\n", + " *\n", + " ACL for RefreshAuthorizationPolicyProtocol, used by the\n", + " dfsadmin and mradmin commands to refresh the security policy in-effect.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.ha.service.protocol.acl\n", + " *\n", + " ACL for HAService protocol used by HAAdmin to manage the\n", + " active and stand-by states of namenode.\n", + " \n", + "\n", + " \n", + " security.zkfc.protocol.acl\n", + " *\n", + " ACL for access to the ZK Failover Controller\n", + " \n", + " \n", + "\n", + " \n", + " security.qjournal.service.protocol.acl\n", + " *\n", + " ACL for QJournalProtocol, used by the NN to communicate with\n", + " JNs when using the QuorumJournalManager for edit logs.\n", + " \n", + "\n", + " \n", + " security.mrhs.client.protocol.acl\n", + " *\n", + " ACL for HSClientProtocol, used by job clients to\n", + " communciate with the MR History Server job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + "\n", + " \n", + " security.resourcetracker.protocol.acl\n", + " *\n", + " ACL for ResourceTrackerProtocol, used by the\n", + " ResourceManager and NodeManager to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcemanager-administration.protocol.acl\n", + " *\n", + " ACL for ResourceManagerAdministrationProtocol, for admin commands.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationclient.protocol.acl\n", + " *\n", + " ACL for ApplicationClientProtocol, used by the ResourceManager\n", + " and applications submission clients to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationmaster.protocol.acl\n", + " *\n", + " ACL for ApplicationMasterProtocol, used by the ResourceManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.containermanagement.protocol.acl\n", + " *\n", + " ACL for ContainerManagementProtocol protocol, used by the NodeManager\n", + " and ApplicationMasters to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.resourcelocalizer.protocol.acl\n", + " *\n", + " ACL for ResourceLocalizer protocol, used by the NodeManager\n", + " and ResourceLocalizer to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.task.protocol.acl\n", + " *\n", + " ACL for TaskUmbilicalProtocol, used by the map and reduce\n", + " tasks to communicate with the parent tasktracker.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.job.client.protocol.acl\n", + " *\n", + " ACL for MRClientProtocol, used by job clients to\n", + " communciate with the MR ApplicationMaster to query job status etc.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " security.applicationhistory.protocol.acl\n", + " *\n", + " ACL for ApplicationHistoryProtocol, used by the timeline\n", + " server and the generic history service client to communicate with each other.\n", + " The ACL is a comma-separated list of user and group names. The user and\n", + " group list is separated by a blank. For e.g. \"alice,bob users,wheel\".\n", + " A special value of \"*\" means all users are allowed.\n", + " \n", + "\n", + " \n", + " \n", + " \n", + " \n", + "\n", + "'''.format((\",\").join(hclients),user,user)\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hadoop-policy.xml','w') as f:\n", + " f.writelines(hadooppolicy)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hadoop-policy.xml {l}:~/hadoop/etc/hadoop/hadoop-policy.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hdfs config, set replication to 1 to cache all the data" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "hdfs_data=\",\".join([f'{l}/{user}/hdfs/data' for l in datadir])\n", + "\n", + "hdfs_site=f'''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " dfs.namenode.secondary.http-address\n", + " {hostname}:50090\n", + " \n", + " \n", + " dfs.namenode.name.dir\n", + " {datadir[0]}/{user}/hdfs/name\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.datanode.data.dir\n", + " {hdfs_data}\n", + " true\n", + " \n", + "\n", + " \n", + " dfs.namenode.checkpoint.dir\n", + " {datadir[0]}/{user}/hdfs/namesecondary\n", + " true\n", + " \n", + " \n", + " dfs.name.handler.count\n", + " 100\n", + " \n", + " \n", + " dfs.blocksize\n", + " 128m\n", + "\n", + " \n", + " dfs.replication\n", + " 1\n", + "\n", + "\n", + "\n", + " dfs.client.read.shortcircuit\n", + " true\n", + "\n", + "\n", + "\n", + " dfs.domain.socket.path\n", + " /var/lib/hadoop-hdfs/dn_socket\n", + "\n", + "\n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/hdfs-site.xml','w') as f:\n", + " f.writelines(hdfs_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/hdfs-site.xml {l}:~/hadoop/etc/hadoop/hdfs-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### mapreduce config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "mapreduce='''\n", + "\n", + "\n", + "\n", + "\n", + "\n", + "\n", + " \n", + " mapreduce.framework.name\n", + " yarn\n", + " \n", + "\n", + " \n", + " mapreduce.job.maps\n", + " 288\n", + " \n", + " \n", + " mapreduce.job.reduces\n", + " 64\n", + " \n", + "\n", + " \n", + " mapreduce.map.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.map.memory.mb\n", + " 6144\n", + " \n", + "\n", + " \n", + " mapreduce.reduce.java.opts\n", + " -Xmx5120M -DpreferIPv4Stack=true\n", + " \n", + " \n", + " mapreduce.reduce.memory.mb\n", + " 6144\n", + " \n", + " \n", + " yarn.app.mapreduce.am.staging-dir\n", + " /user\n", + " \n", + " \n", + " mapreduce.task.io.sort.mb\n", + " 2000\n", + " \n", + " \n", + " mapreduce.task.timeout\n", + " 3600000\n", + " \n", + "\n", + "\n", + " mapreduce.jobhistory.address\n", + " {:s}:10020\n", + "\n", + "\n", + "\n", + "'''.format(hostname)\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/mapred-site.xml','w') as f:\n", + " f.writelines(mapreduce)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/mapred-site.xml {l}:~/hadoop/etc/hadoop/mapred-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### yarn config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [], + "hidden": true + }, + "outputs": [], + "source": [ + "yarn_site=f'''\n", + "\n", + "\n", + " \n", + " yarn.resourcemanager.hostname\n", + " {hostname}\n", + " \n", + " \n", + " yarn.resourcemanager.address\n", + " {hostname}:8032\n", + " \n", + " \n", + " yarn.resourcemanager.webapp.address\n", + " {hostname}:8088\n", + " \n", + " \n", + " yarn.nodemanager.resource.memory-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.nodemanager.resource.cpu-vcores\n", + " {cpu_num}\n", + " \n", + " \n", + " yarn.nodemanager.pmem-check-enabled\n", + " false\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.vmem-check-enabled\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.vmem-pmem-ratio\n", + " 4.1\n", + " \n", + " \n", + " yarn.nodemanager.aux-services\n", + " mapreduce_shuffle,spark_shuffle\n", + " \n", + "\n", + " \n", + " yarn.scheduler.minimum-allocation-mb\n", + " 1024\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-mb\n", + " {mem_mib}\n", + " \n", + " \n", + " yarn.scheduler.minimum-allocation-vcores\n", + " 1\n", + " \n", + " \n", + " yarn.scheduler.maximum-allocation-vcores\n", + " {cpu_num}\n", + " \n", + "\n", + " \n", + " yarn.log-aggregation-enable\n", + " false\n", + " \n", + " \n", + " yarn.nodemanager.log.retain-seconds\n", + " 36000\n", + " \n", + " \n", + " yarn.nodemanager.delete.debug-delay-sec\n", + " 3600\n", + " \n", + " \n", + " yarn.log.server.url\n", + " http://{hostname}:19888/jobhistory/logs/\n", + " \n", + "\n", + " \n", + " yarn.nodemanager.log-dirs\n", + " /home/{user}/hadoop/logs/userlogs\n", + " \n", + " \n", + " yarn.nodemanager.local-dirs\n", + " {sparklocals}\n", + " \n", + " \n", + " \n", + " yarn.nodemanager.aux-services.spark_shuffle.class\n", + " org.apache.spark.network.yarn.YarnShuffleService\n", + " \n", + "\n", + "'''\n", + "\n", + "\n", + "with open(f'/home/{user}/hadoop/etc/hadoop/yarn-site.xml','w') as f:\n", + " f.writelines(yarn_site)\n", + " \n", + "for l in clients:\n", + " !scp ~/hadoop/etc/hadoop/yarn-site.xml {l}:~/hadoop/etc/hadoop/yarn-site.xml >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### hadoop-env" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "#config java home\n", + "if is_arm:\n", + " !echo \"export JAVA_HOME=/usr/lib/jvm/java-8-openjdk-arm64\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "else:\n", + " !echo \"export JAVA_HOME={java_home}\" >> ~/hadoop/etc/hadoop/hadoop-env.sh\n", + "\n", + "for l in clients:\n", + " !scp hadoop/etc/hadoop/hadoop-env.sh {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### workers config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "if clients:\n", + " with open(f'/home/{user}/hadoop/etc/hadoop/workers','w') as f:\n", + " f.writelines(\"\\n\".join(clients))\n", + " for l in clients:\n", + " !scp hadoop/etc/hadoop/workers {l}:~/hadoop/etc/hadoop/ >/dev/null 2>&1\n", + "else:\n", + " !echo {hostname} > ~/hadoop/etc/hadoop/workers" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "### Copy jar from Spark for external shuffle service" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !scp spark/yarn/spark-3.3.1-yarn-shuffle.jar {l}:~/hadoop/share/hadoop/common/lib/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Configure Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "eventlog_dir=f'hdfs://{hostname}:8020/tmp/sparkEventLog'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkconf=f'''\n", + "spark.eventLog.enabled true\n", + "spark.eventLog.dir {eventlog_dir}\n", + "spark.history.fs.logDirectory {eventlog_dir}\n", + "'''\n", + "\n", + "with open(f'/home/{user}/spark/conf/spark-defaults.conf','w+') as f:\n", + " f.writelines(sparkconf)\n", + " \n", + "for l in clients:\n", + " !scp ~/spark/conf/spark-defaults.conf {l}:~/spark/conf/spark-defaults.conf >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "sparkenv = f'export SPARK_LOCAL_DIRS={sparklocals}\\n'\n", + "with open(f'/home/{user}/.bashrc', 'a+') as f:\n", + " f.writelines(sparkenv)\n", + "for l in clients:\n", + " !scp ~/.bashrc {l}:~/.bashrc >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} tail -n10 ~/.bashrc" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Configure startup" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "startup=f'''#!/bin/bash\n", + "echo -1 > /proc/sys/kernel/perf_event_paranoid\n", + "echo 0 > /proc/sys/kernel/kptr_restrict\n", + "echo madvise >/sys/kernel/mm/transparent_hugepage/enabled\n", + "echo 1 > /proc/sys/kernel/numa_balancing\n", + "end=$(($(nproc) - 1))\n", + "for i in $(seq 0 $end); do echo performance > /sys/devices/system/cpu/cpu$i/cpufreq/scaling_governor; done\n", + "for file in $(find /sys/devices/system/cpu/cpu*/power/energy_perf_bias); do echo \"0\" > $file; done\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup', 'w') as f:\n", + " f.writelines(startup)\n", + "\n", + "startup_service=f'''[Unit]\n", + "Description=Configure Transparent Hugepage, Auto NUMA Balancing, CPU Freq Scaling Governor\n", + "\n", + "[Service]\n", + "ExecStart=/usr/local/bin/mystartup.sh\n", + "\n", + "[Install]\n", + "WantedBy=multi-user.target\n", + "'''\n", + "\n", + "with open('/tmp/tmpstartup_service', 'w') as f:\n", + " f.writelines(startup_service)\n", + " \n", + "for l in hclients:\n", + " !scp /tmp/tmpstartup $l:/tmp/tmpstartup\n", + " !scp /tmp/tmpstartup_service $l:/tmp/tmpstartup_service\n", + " !ssh root@$l \"cat /tmp/tmpstartup > /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"chmod +x /usr/local/bin/mystartup.sh\"\n", + " !ssh root@$l \"cat /tmp/tmpstartup_service > /etc/systemd/system/mystartup.service\"\n", + " !ssh $l \"sudo systemctl enable mystartup.service\"\n", + " !ssh $l \"sudo systemctl start mystartup.service\"\n", + " !ssh $l \"sudo systemctl status mystartup.service\"" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "deletable": false, + "editable": false, + "heading_collapsed": true, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "source": [ + "## Inspect CPU Freq & HT" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "if is_arm:\n", + " t = r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"SUBS %0,%0,#1\\n\"\n", + " \"bne 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''\n", + "else:\n", + " t=r'''\n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include \n", + " #include //used for parsing the command line arguments\n", + " #include //used for opening the memory device file\n", + " #include //used for rounding functions\n", + " #include \n", + " #include \n", + " #include \n", + "\n", + " static inline uint64_t GetTickCount()\n", + " {//Return ns counts\n", + " struct timeval tp;\n", + " gettimeofday(&tp,NULL);\n", + " return tp.tv_sec*1000+tp.tv_usec/1000;\n", + " }\n", + "\n", + " uint64_t CNT=CNT_DEF;\n", + "\n", + " int main()\n", + " {\n", + "\n", + " uint64_t start, end;\n", + " start=end=GetTickCount();\n", + "\n", + " asm volatile (\n", + " \"1:\\n\"\n", + " \"dec %0\\n\"\n", + " \"jnz 1b\\n\"\n", + " ::\"r\"(CNT)\n", + " );\n", + "\n", + " end=GetTickCount();\n", + "\n", + " printf(\" total time = %lu, freq = %lu \\n\", end-start, CNT/(end-start)/1000);\n", + "\n", + " return 0;\n", + " }\n", + " '''" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "hidden": true, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "%cd ~\n", + "with open(\"t.c\", 'w') as f:\n", + " f.writelines(t)\n", + "!gcc -O3 -DCNT_DEF=10000000000LL -o t t.c; gcc -O3 -DCNT_DEF=1000000000000LL -o t.delay t.c;\n", + "!for j in `seq 1 $(nproc)`; do echo -n $j; (for i in `seq 1 $j`; do taskset -c $i ./t.delay & done); sleep 1; ./t; killall t.delay; sleep 2; done" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Shutdown Jupyter; source ~/.bashrc; reboot Jupyter; run section [Initialize](#Initialize)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Build gluten" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Install docker" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Instructions from https://docs.docker.com/engine/install/ubuntu/\n", + "\n", + "# Add Docker's official GPG key:\n", + "!sudo -E apt-get update\n", + "!sudo -E apt-get install ca-certificates curl\n", + "!sudo -E install -m 0755 -d /etc/apt/keyrings\n", + "!sudo -E curl -fsSL https://download.docker.com/linux/ubuntu/gpg -o /etc/apt/keyrings/docker.asc\n", + "!sudo chmod a+r /etc/apt/keyrings/docker.asc\n", + "\n", + "# Add the repository to Apt sources:\n", + "!echo \\\n", + " \"deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/docker.asc] https://download.docker.com/linux/ubuntu \\\n", + " $(. /etc/os-release && echo \"$VERSION_CODENAME\") stable\" | \\\n", + " sudo -E tee /etc/apt/sources.list.d/docker.list > /dev/null\n", + "!sudo -E apt-get update" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo -E apt-get install -y docker-ce docker-ce-cli containerd.io docker-buildx-plugin docker-compose-plugin >/dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "if http_proxy or https_proxy:\n", + " !sudo mkdir -p /etc/systemd/system/docker.service.d\n", + " with open('/tmp/http-proxy.conf', 'w') as f:\n", + " s = '''\n", + "[Service]\n", + "{}\n", + "{}\n", + "'''.format(f'Environment=\"HTTP_PROXY={http_proxy}\"' if http_proxy else '', f'Environment=\"HTTPS_PROXY={https_proxy}\"' if https_proxy else '')\n", + " f.writelines(s)\n", + " !sudo cp /tmp/http-proxy.conf /etc/systemd/system/docker.service.d" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!mkdir -p ~/.m2\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "if http_proxy or https_proxy:\n", + " with open(f\"/home/{user}/.m2/settings.xml\",\"w+\") as f:\n", + " f.write('''\n", + "\n", + " ''')\n", + " if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " http_proxy\n", + " true\n", + " http\n", + " {host}\n", + " {port}\n", + " ''')\n", + " if https_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " f.write(f'''\n", + " \n", + " https_proxy\n", + " true\n", + " https\n", + " {host}\n", + " {port}\n", + " ''')\n", + " f.write('''\n", + " \n", + "\n", + "''')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl daemon-reload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo systemctl restart docker.service" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker pull apache/gluten:vcpkg-centos-7" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "container=!sudo docker run -e http_proxy={http_proxy} -e https_proxy={https_proxy} -itd apache/gluten:vcpkg-centos-7\n", + "containerid = container[0]\n", + "containerid" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && git clone https://github.com/apache/incubator-gluten.git gluten\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt && source /opt/rh/devtoolset-9/enable && cd gluten && ./dev/builddeps-veloxbe.sh --enable_vcpkg=ON --enable_hdfs=ON > build.log\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "if os.path.exists(f'/home/{user}/.m2/settings.xml'):\n", + " !sudo docker exec {containerid} bash -c \"mkdir -p ~/.m2\"\n", + " !sudo docker cp /home/{user}/.m2/settings.xml {containerid}:/root/.m2/settings.xml" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker exec {containerid} bash -c \"cd /opt/gluten && mvn clean package -DskipTests -Pspark-3.3 -Pbackends-velox\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!sudo docker cp {containerid}:/opt/gluten/package/target/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar ~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp ~/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar {l}:~/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Generate data" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Build spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian all main\" | sudo tee /etc/apt/sources.list.d/sbt.list\n", + "!echo \"deb https://repo.scala-sbt.org/scalasbt/debian /\" | sudo tee /etc/apt/sources.list.d/sbt_old.list\n", + "!curl -sL \"https://keyserver.ubuntu.com/pks/lookup?op=get&search=0x2EE0EA64E40A89B84B2DF73499E82A75642AC823\" | sudo apt-key add\n", + "!sudo -E apt-get update > /dev/null 2>&1\n", + "!sudo -E apt-get install sbt > /dev/null 2>&1" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "http_proxy=os.getenv('http_proxy')\n", + "https_proxy=os.getenv('https_proxy')\n", + "\n", + "def get_proxy(proxy):\n", + " pos0 = proxy.rfind('/')\n", + " pos = proxy.rfind(':')\n", + " host = http_proxy[pos0+1:pos]\n", + " port = http_proxy[pos+1:]\n", + " return host, port\n", + "\n", + "sbt_opts=''\n", + "\n", + "if http_proxy:\n", + " host, port = get_proxy(http_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttp.proxyHost={host} -Dhttp.proxyPort={port}'\n", + "if https_proxy:\n", + " host, port = get_proxy(https_proxy)\n", + " sbt_opts = f'{sbt_opts} -Dhttps.proxyHost={host} -Dhttps.proxyPort={port}'\n", + " \n", + "if sbt_opts:\n", + " %env SBT_OPTS={sbt_opts}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!git clone https://github.com/databricks/spark-sql-perf.git ~/spark-sql-perf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark-sql-perf && sbt package" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cp ~/spark-sql-perf/target/scala-2.12/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar ~/ipython/" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## Start Hadoop/Spark cluster, Spark history server" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop namenode -format" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/bin/hadoop datanode -format " + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!~/hadoop/sbin/start-dfs.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop dfsadmin -safemode leave" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!hadoop fs -mkdir -p /tmp/sparkEventLog" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!cd ~/spark && sbin/start-history-server.sh" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "master=''\n", + "if clients:\n", + " !~/hadoop/sbin/start-yarn.sh\n", + " master='yarn'\n", + "else:\n", + " # If we run on single node, we use standalone mode\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-slave.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/stop-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-master.sh\n", + " !{os.environ['SPARK_HOME']}/sbin/start-worker.sh spark://{hostname}:7077 -c {cpu_num}\n", + " master=f'spark://{hostname}:7077'" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!jps\n", + "for l in clients:\n", + " !ssh {l} jps" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCH" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpch-dbgen\n", + "!git clone https://github.com/databricks/tpch-dbgen.git ~/tpch-dbgen" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpch-dbgen {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} cd ~/tpch-dbgen && git checkout 0469309147b42abac8857fa61b4cf69a6d3128a8 && make clean && make OS=LINUX" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpch/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions.\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpch_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpch._\n", + "\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dbgenDir = \"/home/{user}/tpch-dbgen\" // location of dbgen\n", + "\n", + "val tables = new TPCHTables(spark.sqlContext,\n", + " dbgenDir = dbgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = false, // do not create the partitioned fact tables\n", + " clusterByPartitionColumns = false, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpch_datagen_parquet=f'''\n", + "cat tpch_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpch_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpch_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpch_datagen_parquet)\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpch_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "## TPCDS" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!rm -rf ~/tpcds-kit\n", + "!git clone https://github.com/databricks/tpcds-kit.git ~/tpcds-kit" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in clients:\n", + " !scp -r ~/tpcds-kit {l}:~/" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "for l in hclients:\n", + " !ssh {l} \"cd ~/tpcds-kit/tools && make clean && make OS=LINUX CC=gcc-9\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "%cd ~/gluten/tools/workload/tpcds/gen_data/parquet_dataset" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Suggest 2x cpu# partitions\n", + "scaleFactor = 1500\n", + "numPartitions = 2*cpu_num if len(clients)==0 else len(clients)*2*cpu_num\n", + "dataformat = \"parquet\" # data format of data source\n", + "dataSourceCodec = \"snappy\"\n", + "rootDir = f\"/tpcds_sf{scaleFactor}_{dataformat}_{dataSourceCodec}\" # root directory of location to create data in." + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "# Verify parameters\n", + "print(f'scaleFactor = {scaleFactor}')\n", + "print(f'numPartitions = {numPartitions}')\n", + "print(f'dataformat = {dataformat}')\n", + "print(f'rootDir = {rootDir}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "scala=f'''import com.databricks.spark.sql.perf.tpcds._\n", + "\n", + "val scaleFactor = \"{scaleFactor}\" // scaleFactor defines the size of the dataset to generate (in GB).\n", + "val numPartitions = {numPartitions} // how many dsdgen partitions to run - number of input tasks.\n", + "\n", + "val format = \"{dataformat}\" // valid spark format like parquet \"parquet\".\n", + "val rootDir = \"{rootDir}\" // root directory of location to create data in.\n", + "val dsdgenDir = \"/home/{user}/tpcds-kit/tools/\" // location of dbgen\n", + "\n", + "val tables = new TPCDSTables(spark.sqlContext,\n", + " dsdgenDir = dsdgenDir,\n", + " scaleFactor = scaleFactor,\n", + " useDoubleForDecimal = false, // true to replace DecimalType with DoubleType\n", + " useStringForDate = false) // true to replace DateType with StringType\n", + "\n", + "\n", + "tables.genData(\n", + " location = rootDir,\n", + " format = format,\n", + " overwrite = true, // overwrite the data that is already there\n", + " partitionTables = true, // create the partitioned fact tables\n", + " clusterByPartitionColumns = true, // shuffle to get partitions coalesced into single files.\n", + " filterOutNullPartitionValues = false, // true to filter out the partition with NULL key value\n", + " tableFilter = \"\", // \"\" means generate all tables\n", + " numPartitions = numPartitions) // how many dsdgen partitions to run - number of input tasks.\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.scala\",\"w\") as f:\n", + " f.writelines(scala)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "executor_cores = 8\n", + "num_executors=cpu_num/executor_cores\n", + "executor_memory = (totalmemory - 10*1024*1024)/num_executors - 1*1024*1024\n", + "\n", + "# Verify parameters\n", + "print(f'--master {master}')\n", + "print(f'--num-executors {int(num_executors)}')\n", + "print(f'--executor-cores {int(executor_cores)}')\n", + "print(f'--executor-memory {int(executor_memory)}k')\n", + "print(f'--conf spark.sql.shuffle.partitions={numPartitions}')\n", + "print(f'--conf spark.sql.parquet.compression.codec={dataSourceCodec}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "tpcds_datagen_parquet=f'''\n", + "cat tpcds_datagen_parquet.scala | {os.environ['SPARK_HOME']}/bin/spark-shell \\\n", + " --master {master} \\\n", + " --name tpcds_gen_parquet \\\n", + " --driver-memory 10g \\\n", + " --num-executors {int(num_executors)} \\\n", + " --executor-cores {int(executor_cores)} \\\n", + " --executor-memory {int(executor_memory)}k \\\n", + " --conf spark.executor.memoryOverhead=1g \\\n", + " --conf spark.sql.broadcastTimeout=4800 \\\n", + " --conf spark.driver.maxResultSize=4g \\\n", + " --conf spark.sql.shuffle.partitions={numPartitions} \\\n", + " --conf spark.sql.parquet.compression.codec={dataSourceCodec} \\\n", + " --conf spark.network.timeout=800s \\\n", + " --conf spark.executor.heartbeatInterval=200s \\\n", + " --jars /home/{user}/ipython/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar \\\n", + "'''\n", + "\n", + "with open(\"tpcds_datagen_parquet.sh\",\"w\") as f:\n", + " f.writelines(tpcds_datagen_parquet)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "hidden": true + }, + "outputs": [], + "source": [ + "!nohup bash tpcds_datagen_parquet.sh" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "heading_collapsed": true + }, + "source": [ + "# Install Trace-Viewer (optional)" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Clone the master branch\n", + "```\n", + "cd ~\n", + "git clone https://github.com/catapult-project/catapult.git -b master\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Trace-Viewer requires python version 2.7. Create a virtualenv for python2.7\n", + "```\n", + "sudo apt install -y python2.7\n", + "virtualenv -p /usr/bin/python2.7 py27\n", + "source py27/bin/activate\n", + "```" + ] + }, + { + "cell_type": "markdown", + "metadata": { + "hidden": true + }, + "source": [ + "Apply patch\n", + "\n", + "```\n", + "cd catapult\n", + "```\n", + "```\n", + "git apply <\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%html\n", + "\n", + "" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# System Settings" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "import pandas as pd\n", + "\n", + "pd.set_option('display.max_rows', None)\n", + "\n", + "# Convert the os.environ object to a dictionary and then to a DataFrame\n", + "env_df = pd.DataFrame(list(dict(os.environ).items()), columns=['Environment Variable', 'Value'])\n", + "\n", + "# Display the DataFrame\n", + "from IPython.display import display\n", + "\n", + "display(env_df)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import socket\n", + "localhost=socket.gethostname()\n", + "local_ip=socket.gethostbyname(localhost)\n", + "\n", + "print(f'localhost: {localhost}')\n", + "print(f'ip: {local_ip}')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "spark_version=!head -n1 $SPARK_HOME/RELEASE | awk '{print $2}'\n", + "spark_version = spark_version[0]\n", + "\n", + "print(f\"Spark version from SPARK_HOME: {spark_version}\")" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import logging\n", + "import sys\n", + "\n", + "logging.basicConfig(format='%(levelname)s : %(message)s', level=logging.ERROR, stream=sys.stdout)\n", + "logger = logging.getLogger()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import os\n", + "\n", + "hdfs_event_dir=''\n", + "local_event_dir=''\n", + "\n", + "def get_spark_eventlog_dir(path):\n", + " eventlog_dir = None\n", + " eventlog_enabled = False\n", + " try:\n", + " with open(path, 'r') as f:\n", + " for line in f:\n", + " if line.startswith('spark.eventLog.dir'):\n", + " eventlog_dir = line.split(' ')[-1].strip()\n", + " elif line.startswith('spark.eventLog.enabled'):\n", + " eventlog_enabled = line.split(' ')[-1].strip().lower() == 'true'\n", + " except FileNotFoundError:\n", + " raise SystemExit(f\"'spark-defaults.conf' not found: {path}\")\n", + " if not eventlog_enabled:\n", + " raise SystemExit(\"'spark.eventLog.enabled' must be enabled.\")\n", + " return eventlog_dir\n", + "\n", + "spark_defaults_conf = None\n", + "\n", + "if 'SPARK_CONF_DIR' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_CONF_DIR'], 'spark-defaults.conf')\n", + "elif 'SPARK_HOME' in os.environ:\n", + " spark_defaults_conf = os.path.join(os.environ['SPARK_HOME'], 'conf', 'spark-defaults.conf')\n", + "\n", + "if spark_defaults_conf:\n", + " event_log_dir = get_spark_eventlog_dir(spark_defaults_conf)\n", + " if event_log_dir:\n", + " print(f\"spark.eventLog.dir: {event_log_dir}\")\n", + " if event_log_dir[:7] == 'hdfs://':\n", + " hdfs_event_dir = event_log_dir\n", + " elif event_log_dir[:6] == 'file:/':\n", + " local_event_dir = event_log_dir[6:]\n", + " else:\n", + " raise SystemExit(f\"'spark.eventLog.dir' is not configured in {spark_defaults_conf}\")\n", + "else:\n", + " raise SystemExit(\"Cannot get `spark.eventLog.dir`. Neither SPARK_CONF_DIR nor SPARK_HOME defined in envrionment variables.\")\n", + " " + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Monitor" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import findspark\n", + "import os\n", + "\n", + "findspark.init(os.environ['SPARK_HOME'])\n", + "os.environ.setdefault('SPARK_SUBMIT_OPTS', '-Dscala.usejavacp=true')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import warnings\n", + "warnings.filterwarnings('ignore')\n", + "\n", + "import atexit\n", + "import collections\n", + "import gzip\n", + "import importlib\n", + "import json\n", + "import logging\n", + "import math\n", + "import os\n", + "import pathlib\n", + "import shutil\n", + "import signal\n", + "import subprocess\n", + "import tempfile\n", + "import threading\n", + "import time\n", + "import timeit\n", + "import traceback\n", + "\n", + "import matplotlib\n", + "import matplotlib.colors as colors\n", + "import matplotlib.pyplot as plt\n", + "import matplotlib.ticker as mtick\n", + "import numpy as np\n", + "import pandas as pd\n", + "import platform\n", + "import pyspark\n", + "import pyspark.sql.functions as F\n", + "import pyspark.sql.types as T\n", + "import spylon_kernel\n", + "from collections import namedtuple\n", + "from concurrent.futures import ThreadPoolExecutor\n", + "from datetime import date\n", + "from functools import reduce\n", + "from IPython.display import display, HTML\n", + "from matplotlib import rcParams\n", + "from pyspark import SparkConf, SparkContext\n", + "from pyspark.ml import Pipeline\n", + "from pyspark.ml.feature import StringIndexer, VectorAssembler\n", + "from pyspark.sql import SparkSession, SQLContext, Window\n", + "from pyspark.sql.functions import col, floor, lit, rank, to_date\n", + "from pyspark.sql.types import (DoubleType, FloatType, IntegerType,\n", + " StringType, StructField, StructType,\n", + " TimestampType)\n", + "\n", + "from spylon_kernel import register_ipython_magics\n", + "from spylon.spark.utils import SparkJVMHelpers\n", + "\n", + "register_ipython_magics()\n", + "\n", + "rcParams['font.sans-serif'] = 'Courier New'\n", + "rcParams['font.family'] = 'Courier New'\n", + "rcParams['font.size'] = '12'\n", + "\n", + "%matplotlib inline\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import socket\n", + "import os\n", + "import sys\n", + "\n", + "from pathlib import Path\n", + "home = str(Path.home())\n", + "\n", + "def upload_profile(server, base_dir, appid):\n", + " local_profile_dir = os.path.join(home, 'profile')\n", + " !mkdir -p {local_profile_dir}\n", + " !cd {local_profile_dir}; rm -f {appid}.tar.gz; tar zcvf {appid}.tar.gz {appid} >/dev/null 2>&1\n", + " \n", + " server_local_dir=os.path.join('PAUS', base_dir)\n", + " server_local_profile_dir=os.path.join(server_local_dir, 'profile')\n", + " server_hdfs_dir=f'/{base_dir}/'\n", + "\n", + " !ssh {server} \"mkdir -p {server_local_profile_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid} >/dev/null 2>&1\"\n", + " !scp {local_profile_dir}/{appid}.tar.gz {server}:{server_local_profile_dir}/\n", + " !ssh {server} \"cd {server_local_profile_dir} && tar zxf {appid}.tar.gz\"\n", + " !ssh {server} \"hdfs dfs -mkdir -p {server_hdfs_dir}; hdfs dfs -rm -r {server_hdfs_dir}{appid}; hdfs dfs -put {server_local_profile_dir}/{appid} {server_hdfs_dir}\"\n", + " !ssh {server} \"cd {server_local_profile_dir}; rm {appid}.tar.gz; rm -r {appid}\"\n", + "\n", + "def killsar(clients):\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w sar | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w pidstat | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + " for l in clients:\n", + " out=!ssh $l \"ps aux | grep -w perf | grep -v grep | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh root@$l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def killnumactl(clients):\n", + " for l in clients:\n", + " out =!ssh $l \"ps aux | grep numactl | grep bash | tr -s ' ' | cut -d' ' -f2\"\n", + " for x in out:\n", + " !ssh $l \"kill $x > /dev/null 2>&1\"\n", + "\n", + "def startmonitor(clients,appid,**kwargs):\n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + " \n", + " for l in clients:\n", + " !ssh root@{l} date\n", + " \n", + " killsar(clients)\n", + " \n", + " perfsyscalls=kwargs.get(\"collect_perf_syscall\",None)\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !mkdir -p {prof_client}\n", + " !ssh {l} mkdir -p {prof_client}\n", + " !ssh {l} \"sar -o {prof_client}/sar.bin -r -u -d -B -n DEV 1 >/dev/null 2>&1 &\"\n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " if kwargs.get(\"collect_pid\",False):\n", + " !ssh {l} \"jps | grep CoarseGrainedExecutorBackend | head -n 1 | cut -d' ' -f 1 | xargs -I % pidstat -h -t -p % 1 > {prof_client}/pidstat.out 2>/dev/null &\"\n", + " !ssh root@{l} 'cat /proc/uptime | cut -d\" \" -f 1 | xargs -I ^ date -d \"- ^ seconds\" +%s.%N' > $prof/$l/uptime.txt\n", + " if kwargs.get(\"collect_sched\",False):\n", + " !ssh root@{l} 'perf trace -e \"sched:sched_switch\" -C 8-15 -o {prof_client}/sched.txt -T -- sleep 10000 >/dev/null 2>/dev/null &'\n", + " if perfsyscalls is not None:\n", + " !ssh root@{l} \"perf stat -e 'syscalls:sys_exit_poll,syscalls:sys_exit_epoll_wait' -a -I 1000 -o {prof_client}/perfstat.txt >/dev/null 2>&1 & \"\n", + " if kwargs.get(\"collect_hbm\",False):\n", + " hbm_nodes = kwargs.get(\"hbm_nodes\")\n", + " if hbm_nodes is not None:\n", + " print(\"collect_hbm\")\n", + " hbm_nodes = '\\|'.join([\"node \" + str(i) for i in hbm_nodes])\n", + " %env hbm_numa_nodes={hbm_nodes}\n", + " %env hbm_l = {l}\n", + " %env hbm_prof = {prof}\n", + " !ssh $hbm_l \"echo timestamp, size, free > $hbm_prof/$hbm_l/numactl.csv\"\n", + " !ssh $hbm_l \"while :; do echo \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'size' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }'), \\$(numactl -H | grep '$hbm_numa_nodes' | grep 'free' | awk '{ print \\$4 }' | awk '{ s += \\$1 } END { print s }') | ts '%Y-%m-%d %H:%M:%S,' >> $hbm_prof/$hbm_l/numactl.csv; sleep 1; done >/dev/null 2>&1 &\"\n", + " else:\n", + " print(\"Missing argument: hbm_nodes. e.g. hbm_nodes = list(range(8,16))\")\n", + " return prof\n", + "\n", + "def stopmonitor(clients, sc, appid, **kwargs):\n", + " %cd ~\n", + " \n", + " local_profile_dir=os.path.join(home, 'profile')\n", + " prof=os.path.join(local_profile_dir, appid)\n", + " !mkdir -p {prof}\n", + "\n", + " killsar(clients)\n", + " killnumactl(clients) \n", + " \n", + " with open(f\"{prof}/starttime\",\"w\") as f:\n", + " f.write(\"{:d}\".format(int(time.time()*1000)))\n", + " \n", + " for l in clients:\n", + " prof_client=os.path.join(prof, l)\n", + " !ssh {l} \"sar -f {prof_client}/sar.bin -r > {prof_client}/sar_mem.sar;sar -f {prof_client}/sar.bin -u > {prof_client}/sar_cpu.sar;sar -f {prof_client}/sar.bin -d -p > {prof_client}/sar_disk.sar;sar -f {prof_client}/sar.bin -n DEV > {prof_client}/sar_nic.sar;sar -f {prof_client}/sar.bin -B > {prof_client}/sar_page.sar;\" \n", + " !ssh root@{l} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f 1 | xargs -I % bash -c '(cat /proc/%/status >> {prof_client}/%.stat; cat /proc/%/io >> {prof_client}/%.stat)'\"\n", + " !ssh {l} \"sar -V \" > {prof_client}/sarv.txt\n", + " !test -f {prof_client}/perfstat.txt && head -n 1 {prof_client}/perfstat.txt > {prof_client}/perfstarttime\n", + " if l!= socket.gethostname():\n", + " !scp -r {l}:{prof_client} {prof}/ > /dev/null 2>&1\n", + " \n", + " if sc is not None:\n", + " sc.stop()\n", + " \n", + " if hdfs_event_dir != '':\n", + " !hadoop fs -copyToLocal {hdfs_event_dir}/{appid} {prof}/app.log\n", + " elif local_event_dir != '':\n", + " !cp {local_event_dir}/{appid} {prof}/app.log" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def pinexecutor_numa(clients):\n", + " cpunum = !ssh {clients[0]} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum = int(cpunum[0])\n", + " \n", + " numanodes=!ssh {clients[0]} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes = list(filter(lambda x: x != '', numanodes))\n", + " print(numanodes)\n", + " for client in clients:\n", + " pids=!ssh {client} \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " cpunum_c = !ssh {client} \"grep 'processor' /proc/cpuinfo | wc -l\"\n", + " cpunum_c = int(cpunum_c[0])\n", + " if cpunum_c != cpunum:\n", + " print(f\"client {client} cpunum not match!\")\n", + " return\n", + " numanodes_c=!ssh {client} \"cat /sys/devices/system/node/node*/cpulist\"\n", + " numanodes_c = list(filter(lambda x: x != '', numanodes))\n", + " time.sleep(1)\n", + " print(numanodes_c)\n", + " if numanodes_c != numanodes:\n", + " print(f\"client {client} numanodes not match!\")\n", + " return\n", + " \n", + " idx = 0\n", + " nodes=len(numanodes)\n", + " for i in range(nodes):\n", + " cpus = numanodes[i]\n", + " for l in pids[idx:idx+int(len(pids)/nodes)]: # executors on 1 numanode\n", + " print(f\" {cpus} {l}\")\n", + " !ssh {client} \"taskset -a -p -c $cpus $l > /dev/null 2>&1 \"\n", + " idx += int(len(pids)/nodes)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_pagecache(clients, run_gluten=True):\n", + " for l in clients:\n", + " if run_gluten:\n", + " !ssh root@$l \"echo 80 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 50 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 360000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_writeback_centisecs\"\n", + "\n", + " else:\n", + " !ssh root@$l \"echo 10 > /proc/sys/vm/dirty_ratio\"\n", + " !ssh root@$l \"echo 20 > /proc/sys/vm/dirty_background_ratio\"\n", + " !ssh root@$l \"echo 3000 > /proc/sys/vm/dirty_expire_centisecs\"\n", + " !ssh root@$l \"echo 500 > /proc/sys/vm/dirty_writeback_centisecs\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def print_kernel_params(clietns):\n", + " params = {\n", + " 'transparent hugepage': '/sys/kernel/mm/transparent_hugepage/enabled',\n", + " 'auto numa balancing': '/proc/sys/kernel/numa_balancing',\n", + " 'scaling governor': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_governor',\n", + " 'scaling max freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_max_freq',\n", + " 'scaling cur freq': '/sys/devices/system/cpu/cpu*/cpufreq/scaling_cur_freq',\n", + " 'power & perf policy': '/sys/devices/system/cpu/cpu*/power/energy_perf_bias',\n", + " 'dirty_ratio': '/proc/sys/vm/dirty_ratio',\n", + " 'dirty_background_ratio': '/proc/sys/vm/dirty_background_ratio',\n", + " 'dirty_expire_centisecs': '/proc/sys/vm/dirty_expire_centisecs',\n", + " 'dirty_writeback_centisecs': '/proc/sys/vm/dirty_writeback_centisecs'\n", + " }\n", + " for k, param in params.items():\n", + " print()\n", + " print(f'{k} ({param})')\n", + " for l in clients:\n", + " print(l + \": \", end='')\n", + " res = !ssh root@$l \"cat {param}\"\n", + " print(*res)\n", + " # print numactl\n", + " print()\n", + " print(\"numactl -H\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res = !ssh $l \"numactl -H\"\n", + " print('\\n'.join(res))\n", + " # print memory freq\n", + " print()\n", + " print(\"Memory Frequency\")\n", + " for l in clients:\n", + " print(l + \":\")\n", + " res= !ssh root@$l \"dmidecode -t memory | grep Speed\"\n", + " print('\\n'.join(res))" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "def dropcache(clients):\n", + " for l in clients:\n", + " !ssh root@$l \"sync && echo 3 > /proc/sys/vm/drop_caches; echo 1 >/proc/sys/vm/compact_memory; free -h\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def config_mem_cgroup(clients):\n", + " mem_cgroup = \"\"\"\n", + "CGROUP_ROOT=/sys/fs/cgroup/gluten\n", + "\n", + "if [ ! -d $CGROUP_ROOT ] ; then\n", + " sudo mkdir $CGROUP_ROOT\n", + " # enable memory for subtree\n", + " sudo bash -c \"echo '+memory' >> $CGROUP_ROOT/cgroup.subtree_control\"\n", + "fi\n", + "\n", + "# move each process to sub memory group\n", + "index=0\n", + "for pid in `jps | grep Coarse | awk '{print $1}'` ; do\n", + " target_cgroup=$CGROUP_ROOT/mem-${index}\n", + " if [ ! -d $target_cgroup ] ; then\n", + " sudo mkdir $target_cgroup\n", + " fi\n", + " proc_file=$target_cgroup/cgroup.procs\n", + " sudo bash -c \"echo $pid >> $proc_file\"\n", + " index=`expr $index + 1`\n", + "done\n", + " \"\"\"\n", + " with open(f'{home}/mem-cgroup.sh', 'w+') as f:\n", + " f.writelines(mem_cgroup)\n", + " for l in clients:\n", + " !scp {home}/mem-cgroup.sh {l}:{home}/ >/dev/null 2>&1\n", + " !ssh {l} \"bash {home}/mem-cgroup.sh >/dev/null 2>&1 &\"" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "import pandas as pd\n", + "import matplotlib.pyplot as plt\n", + "import os\n", + "\n", + "\n", + "from IPython.display import display, HTML\n", + "\n", + "def get_io_stats(appid, client):\n", + " file_path = os.path.join(home,'profile',appid,client)\n", + " statf = [f for f in os.listdir(file_path) if f.endswith('.stat')]\n", + " statmap=[]\n", + " for f in statf:\n", + " statmap.append({'pid':f[:-len(\".stat\")]})\n", + " with open(os.path.join(file_path, f),\"r\") as fi:\n", + " cnts=fi.readlines()\n", + " for l in cnts:\n", + " for fld in ['rchar','wchar','syscr','syscw','read_bytes','write_bytes','cancelled_write_bytes']:\n", + " if l.startswith(fld):\n", + " if not fld in statmap[-1]:\n", + " statmap[-1][fld]=int(l.split(\" \")[-1].strip())\n", + " else:\n", + " statmap[-1][fld]=(int(l.split(\" \")[-1].strip())-statmap[-1][fld])/1024/1024/1024\n", + "\n", + " df = pd.DataFrame(statmap).drop('pid', axis=1).sum().to_frame()\n", + " df.columns = ['sum']\n", + " return df\n", + "\n", + "# Preprocess 'time' column\n", + "def process_time(dataframes):\n", + " for df in dataframes:\n", + " df.columns=['time']+list(df.columns[1:])\n", + " df = df[df.time != 'Average:']\n", + " df['time'] = pd.to_datetime(df['time'], format='%H:%M:%S').dt.time\n", + " df['time'] = df['time'].apply(lambda dt: dt.hour*3600 + dt.minute*60 + dt.second)\n", + "\n", + " offset = 12 * 3600 # half-day seconds\n", + " for i in range(1, len(df)):\n", + " if df['time'].iloc[i] < df['time'].iloc[i-1]: # Detect AM->PM or PM->AM\n", + " for j in range(i, len(df)): # Apply offset until end\n", + " df['time'].iloc[j] += offset\n", + "\n", + " df['time'] = df['time'].astype(int)\n", + " yield df\n", + "\n", + "def draw_sar(appid, qtime=None, disk_dev=None, nic_dev=None, client=None):\n", + " if client is None:\n", + " client = clients[0]\n", + "\n", + " display(HTML('{:s}'.format(client)))\n", + "\n", + " display(get_io_stats(appid, client))\n", + "\n", + " # Read data\n", + " profile_dir = os.path.join(home,'profile',appid,client)\n", + " datafiles = [os.path.join(profile_dir, datafile) for datafile in ['sar_cpu.sar', 'sar_mem.sar', 'sar_disk.sar', 'sar_nic.sar', 'sar_page.sar']]\n", + " dataframes = [pd.read_csv(datafile, header=1, delim_whitespace=True, parse_dates=True) for datafile in datafiles]\n", + " \n", + " num_figs=5\n", + " fig, axs=plt.subplots(num_figs,1,sharex=True,figsize=(30,5*4))\n", + "\n", + " [cpu_df, mem_df, disk_df, nic_df, page_df] = process_time(dataframes)\n", + "\n", + " # CPU usage\n", + " cpu_df['total'] = cpu_df['%user'] + cpu_df['%system'] + cpu_df['%iowait']\n", + "\n", + " starttime = cpu_df[cpu_df['total'] > 50]['time'].min() - 1\n", + " cpu_df['time'] -= starttime\n", + "\n", + " axs[4].stackplot(cpu_df['time'], cpu_df['%user'], cpu_df['%system'], cpu_df['%iowait'], labels=['user','system','iowait'])\n", + " axs[4].legend(loc='upper left')\n", + "\n", + " # Memory usage\n", + " mem_df['dirty_cached'] = mem_df['kbdirty'] * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['clean_cached'] = (mem_df['kbcached'] - mem_df['kbdirty']) * mem_df['%memused'] / mem_df['kbmemused']\n", + " mem_df['used'] = mem_df['kbmemused'] * mem_df['%memused'] / mem_df['kbmemused']\n", + "# mem_df['used'] = (mem_df['kbmemused'] - mem_df['kbbuffers'] - mem_df['kbcached'])* mem_df['%memused'] / mem_df['kbmemused']\n", + "\n", + " mem_df['time'] -= starttime\n", + "\n", + " axs[0].stackplot(mem_df['time'], mem_df['used'], mem_df['clean_cached'], mem_df['dirty_cached'], labels=['used','clean cached','dirty cached'])\n", + " axs[0].legend(loc='upper left')\n", + " axs[0].grid(axis = 'y')\n", + "\n", + " # Disk usage\n", + " if disk_dev is not None:\n", + " disk_df = disk_df[disk_df['DEV'].isin(disk_dev)]\n", + " disk_df['rkB/s'] = disk_df['rkB/s'].astype(float)\n", + " disk_df['wkB/s'] = disk_df['wkB/s'].astype(float)\n", + " disk_df['%util'] = disk_df['%util'].astype(float)\n", + "\n", + "\n", + " disk_df = disk_df.groupby('time').agg({'rkB/s': 'sum', 'wkB/s': 'sum', '%util':'mean'}).reset_index()\n", + " disk_df['read'] = disk_df['rkB/s'] / 1024\n", + " disk_df['write'] = disk_df['wkB/s'] / 1024\n", + "\n", + " disk_df['time'] -= starttime\n", + "\n", + " axs[1].stackplot(disk_df['time'], disk_df['read'], disk_df['write'], labels=['read MB/s','write MB/s'])\n", + " axs[1].grid(axis = 'y')\n", + "\n", + " ax2 = axs[1].twinx()\n", + "\n", + " ax2.plot(disk_df['time'], disk_df['%util'],'g-')\n", + " axs[1].legend(loc='upper left')\n", + "\n", + " \n", + " # Nic usage\n", + " if nic_dev is not None:\n", + " nic_df = nic_df[nic_df['IFACE'].isin(nic_dev)]\n", + " nic_df['rxkB/s'] = nic_df['rxkB/s'].astype(float)\n", + " nic_df['txkB/s'] = nic_df['txkB/s'].astype(float)\n", + " \n", + " nic_df = nic_df.groupby('time').agg({'rxkB/s': 'sum', 'txkB/s': \"sum\"}).reset_index()\n", + " nic_df['rx'] = nic_df['rxkB/s'] / 1024\n", + " nic_df['tx'] = nic_df['txkB/s'] / 1024\n", + " \n", + " nic_df['time'] -= starttime\n", + " \n", + " axs[2].stackplot(nic_df['time'], nic_df['rx'], nic_df['tx'], labels=['rx MB/s','tx MB/s'])\n", + " axs[2].legend(loc='upper left')\n", + " axs[2].grid(axis = 'y')\n", + "\n", + " # Pagefaults\n", + " page_df['minflt/s'] = page_df['fault/s'] - page_df['majflt/s']\n", + " \n", + " page_df['time'] -= starttime\n", + "\n", + " axs[3].stackplot(page_df['time'], page_df['minflt/s'], page_df['majflt/s'], labels=['minor_fault/s','major_fault/s'])\n", + " axs[3].legend(loc='upper left')\n", + " axs[3].grid(axis = 'y')\n", + "\n", + " # Add vertical lines and text for qtime, and calculate per query cpu%\n", + " if qtime is not None:\n", + " for ax in axs:\n", + " x = 0\n", + " ax.axvline(x = x, color = 'b')\n", + " for k, v in qtime.items():\n", + " x += v\n", + " ax.axvline(x = x, color = 'b')\n", + "\n", + " tx = 0\n", + " for k, v in qtime.items():\n", + " if v / x > 15 / 772:\n", + " ax.text(tx + v / 2 - 6 * x / 772, ax.get_ylim()[1] * 1.05, k)\n", + " tx += v\n", + "\n", + " x = 0\n", + " qtime_se = {}\n", + " cols = ['%user','%system','%iowait']\n", + " for k, v in qtime.items():\n", + " filtered_df = cpu_df[(cpu_df['time'] >= x) & (cpu_df['time'] <= x+v)]\n", + " averages = filtered_df[cols].mean()\n", + " qtime_se[k] = averages.tolist()\n", + " x += v\n", + " if qtime_se:\n", + " perqcpu = pd.DataFrame(qtime_se).T\n", + " perqcpu.columns = cols\n", + " display(perqcpu)\n", + "\n", + " plt.show()\n" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def convert_to_etc_gmt(tz_offset=None):\n", + " # Run the 'date +%z' command and get the output\n", + " if not tz_offset:\n", + " tz_offset = !date +%z\n", + " tz_offset = tz_offset[0]\n", + " \n", + " # Extract the sign and the hour/minute offset\n", + " sign = tz_offset[0]\n", + " hours = int(tz_offset[1:3])\n", + " minutes = int(tz_offset[3:])\n", + "\n", + " # Convert the offset to a GMT value\n", + " gmt_offset = hours + (minutes / 60)\n", + " if sign == '+':\n", + " gmt_offset = -gmt_offset\n", + " else:\n", + " gmt_offset = abs(gmt_offset)\n", + "\n", + " # Construct the Etc/GMT string\n", + " etc_gmt = f\"Etc/GMT{int(gmt_offset):+d}\"\n", + " return etc_gmt" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# TestTPC" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "import socket\n", + "from dataclasses import dataclass\n", + "from functools import wraps\n", + "from pathlib import Path\n", + "from typing import List \n", + "\n", + "class TestTPC:\n", + " @dataclass\n", + " class query_info:\n", + " tables: List[str]\n", + " sql: List[str]\n", + "\n", + " query_infos = {}\n", + " query_ids =[]\n", + "\n", + " tpctables=[]\n", + " tpc_query_path = ''\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, workload, server, base_dir, nb_name, data_source = 'parquet'):\n", + " self.spark = spark\n", + " self.sc = spark.sparkSession.sparkContext\n", + " self.appid = self.sc.applicationId\n", + " self.run_gluten = run_gluten\n", + " self.workload = workload\n", + " self.table_dir = table_dir\n", + " self.server = server\n", + " self.base_dir = base_dir\n", + " self.nb_name = nb_name\n", + " self.data_source = data_source\n", + " self.table_loaded = False\n", + " self.result = {}\n", + " self.stopped = False\n", + " self.perf_html = ''\n", + " self.finished_nb = ''\n", + " for l in os.listdir(self.tpc_query_path):\n", + " if (l[-3:] == 'sql'):\n", + " with open(self.tpc_query_path+l,\"r\") as f:\n", + " self.query_infos[l.split(\".\")[0]]=self.query_info(self.tpctables,[\"\\n\".join(f.readlines())])\n", + " self.query_ids = sorted(self.query_infos.keys(), key=lambda x: str(len(x))+x if x[-1] != 'a' and x[-1] != 'b' else str(len(x)-1) + x)\n", + " print(\"http://{}:18080/history/{}/jobs/\".format(local_ip, self.sc.applicationId))\n", + " \n", + " def start_monitor(self, clients, **kw):\n", + " startmonitor(clients, self.appid, **kw)\n", + " \n", + " def stop_monitor(self, clients, **kw):\n", + " if self.stopped:\n", + " return\n", + " stopmonitor(clients, self.sc, self.appid, **kw)\n", + " if self.server:\n", + " output_nb = f'{self.nb_name[:-6]}-{self.appid}.ipynb'\n", + " if output_nb.startswith(home):\n", + " output_nb_name = os.path.relpath(output_nb, home)\n", + " else:\n", + " output_nb_name = output_nb\n", + " output_nb_dir = os.path.dirname(output_nb_name)\n", + " server_nb_dir = os.path.join('PAUS', self.base_dir, output_nb_dir)\n", + " !ssh {self.server} \"mkdir -p {server_nb_dir}\"\n", + " !scp {output_nb} {self.server}:{server_nb_dir}\n", + " self.finished_nb = f\"http://{self.server}:8888/tree/{self.base_dir}/{output_nb_name}\"\n", + " self.stopped = True\n", + "\n", + " def run_perf_analysis(self, disk_dev, nic_dev):\n", + " if not self.server:\n", + " return\n", + "\n", + " upload_profile(self.server, self.base_dir, self.appid)\n", + "\n", + " ts=time.strftime(\"%Y_%m_%d_%H%M%S\")\n", + " name=f'{self.workload}_gluten' if self.run_gluten else f'{self.workload}_vanilla'\n", + " run_script=f'{gluten_home}/tools/workload/benchmark_velox/analysis/run_perf_analysis.sh'\n", + " \n", + " disk=','.join(disk_dev)\n", + " nic=','.join(nic_dev)\n", + "\n", + " command =' '.join(['bash', run_script, '--ts', ts, '--base-dir', self.base_dir, '--name', name, '--appid', self.appid, '--disk', disk, '--nic', nic, '--tz', convert_to_etc_gmt()])\n", + " print(command)\n", + "\n", + " # Block if running on local cluster.\n", + " if self.server == localhost:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1\"\n", + " else:\n", + " !ssh {self.server} \"{command} > /dev/null 2>&1 &\"\n", + "\n", + " self.perf_html=f'http://{self.server}:8888/view/{self.base_dir}/html/{ts}_{name}_{self.appid}.html'\n", + " display(HTML(f'{self.perf_html}'))\n", + " \n", + " def load_table(self, table):\n", + " if type(self.table_dir)==list:\n", + " return self.spark.read.format(self.data_source).load([os.path.join(t, table) for t in self.table_dir])\n", + " else:\n", + " return self.spark.read.format(self.data_source).load(os.path.join(self.table_dir, table))\n", + " \n", + " def load_tables_as_tempview(self, tables):\n", + " for table in tables:\n", + " df = self.load_table(table)\n", + " df.createOrReplaceTempView(table)\n", + " \n", + " def load_all_tables_as_tempview(self):\n", + " print(f\"Loading all tables: {self.tpctables}\")\n", + " self.load_tables_as_tempview(self.tpctables)\n", + " \n", + " def load_query(self, query):\n", + " info = self.query_infos[query]\n", + " return [self.spark.sql(q) for q in info.sql]\n", + " \n", + " def run_query(self, query, explain = False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " start_time = timeit.default_timer()\n", + " print(\"start query \" + query + \", application id \" + self.sc.applicationId)\n", + " print(\"{} : {}\".format(\"Start time\", start_time))\n", + " self.sc.setJobDescription(query)\n", + "\n", + " queries = self.load_query(query)\n", + " for q in queries:\n", + " if explain: q.explain()\n", + " collect=q.collect()\n", + " end_time = timeit.default_timer()\n", + " duration = end_time - start_time\n", + " display(HTML(('Completed Query. Time(sec): {:f}'.format(duration))))\n", + " \n", + " self.result[query] = duration\n", + " if print_result:\n", + " print(collect)\n", + "\n", + " def power_run(self, explain=False, print_result=False, load_table=True):\n", + " if load_table:\n", + " self.load_all_tables_as_tempview()\n", + " for l in self.query_ids:\n", + " self.run_query(l, explain=explain, print_result=print_result, load_table=False)\n", + "\n", + " def print_result(self):\n", + " print(self.result)\n", + " print()\n", + " durations = [float(i) for i in self.result.values()]\n", + " print(\"total duration:\")\n", + " print(sum(durations))\n", + " print()\n", + " if self.server:\n", + " print(self.finished_nb)\n", + " print(f\"http://{self.server}:1088/tracing_examples/trace_viewer.html#/tracing/test_data/{self.appid}.json\")\n", + " print(f\"http://{self.server}:18080/history/{self.appid}\")\n", + " print(self.perf_html)\n", + " print(self.appid)\n", + " for i in durations:\n", + " print(i)\n", + " \n", + "class TestTPCH(TestTPC):\n", + " tpctables = ['customer', 'lineitem', 'nation', 'orders', 'part', 'partsupp', 'region', 'supplier']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpch-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpch', server, base_dir, nb_name, data_source)\n", + " \n", + "class TestTPCDS(TestTPC):\n", + " tpctables = [ 'call_center',\n", + " 'catalog_page',\n", + " 'catalog_returns',\n", + " 'catalog_sales',\n", + " 'customer',\n", + " 'customer_address',\n", + " 'customer_demographics',\n", + " 'date_dim',\n", + " 'household_demographics',\n", + " 'income_band',\n", + " 'inventory',\n", + " 'item',\n", + " 'promotion',\n", + " 'reason',\n", + " 'ship_mode',\n", + " 'store',\n", + " 'store_returns',\n", + " 'store_sales',\n", + " 'time_dim',\n", + " 'warehouse',\n", + " 'web_page',\n", + " 'web_returns',\n", + " 'web_sales',\n", + " 'web_site']\n", + " tpc_query_path = f'{gluten_home}/tools/gluten-it/common/src/main/resources/tpcds-queries/'\n", + " \n", + " def __init__(self, spark, table_dir, run_gluten, server, base_dir, nb_name, data_source = 'parquet'):\n", + " TestTPC.__init__(self,spark, table_dir, run_gluten, 'tpcds', server, base_dir, nb_name, data_source)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Create SparkContext" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## default config" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "import os\n", + "\n", + "def findjemalloc():\n", + " l = clients[0]\n", + " jemallocDir = !ssh $l \"whereis libjemalloc.so.2\"\n", + " libjemalloc = jemallocDir[0].split(' ')\n", + " return libjemalloc[1]\n", + "\n", + "def get_py4jzip():\n", + " spark_home=os.environ['SPARK_HOME']\n", + " py4jzip = !ls {spark_home}/python/lib/py4j*.zip\n", + " return py4jzip[0]\n", + "\n", + "def default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars='', app_name='', master='yarn', run_gluten=False):\n", + " # Create a temp directory that gets cleaned up on exit\n", + " output_dir = os.path.abspath(tempfile.mkdtemp())\n", + " def cleanup():\n", + " shutil.rmtree(output_dir, True)\n", + " atexit.register(cleanup)\n", + " signal.signal(signal.SIGTERM, cleanup)\n", + "\n", + "##################################################\n", + " def convert_to_bytes(size):\n", + " units = {'k': 1, 'm': 2, 'g': 3}\n", + " size = size.lower()\n", + " if size[-1] in units:\n", + " return int(size[:-1]) * 1024 ** units[size[-1]]\n", + " else:\n", + " return int(size)\n", + "\n", + " def yarn_padding(size):\n", + " min_size = convert_to_bytes('1g')\n", + " step = min_size\n", + " while size > min_size:\n", + " min_size += step\n", + " return min_size - size\n", + " \n", + " num_nodes = len(clients)\n", + " num_executors = num_nodes*executors_per_node\n", + " parallelism = num_executors*cores_per_executor*task_per_core\n", + "\n", + " if run_gluten:\n", + " offheap_ratio = gluten_offheap_ratio\n", + " else:\n", + " offheap_ratio = vanilla_offheap_ratio\n", + " driver_memory = convert_to_bytes('20g')\n", + " executor_memory_overhead = convert_to_bytes('1g')\n", + " \n", + " # Minimun executor memory\n", + " min_memory = convert_to_bytes('1g')\n", + "\n", + " # Calculate executor onheap memory\n", + " num_driver = 1 if localhost in clients else 0\n", + " executor_memory = math.floor((convert_to_bytes(memory_per_node) - (executor_memory_overhead + min_memory)*executors_per_node - (driver_memory + min_memory)*num_driver)/(offheap_ratio*num_driver + (1+offheap_ratio)*executors_per_node))\n", + " executor_memory = max(executor_memory, min_memory)\n", + " # Calculate driver/executor offheap memory in MB\n", + " #offheap_memory_per_node = convert_to_bytes(memory_per_node) - (executor_memory + executor_memory_overhead) * executors_per_node\n", + " if offheap_ratio > 0:\n", + " enable_offheap = True\n", + " offheap_memory = math.floor(executor_memory*offheap_ratio)\n", + " else:\n", + " enable_offheap = False\n", + " offheap_memory = 0\n", + "\n", + " byte_to_mb = lambda x: int(x/(1024 ** 2))\n", + " driver_memory_mb = byte_to_mb(driver_memory)\n", + " executor_memory_overhead_mb = byte_to_mb(executor_memory_overhead)\n", + " executor_memory_mb = byte_to_mb(executor_memory)\n", + " offheap_memory_mb = byte_to_mb(offheap_memory)\n", + " \n", + " executor_totalmem_mb = executor_memory_overhead_mb + executor_memory_mb + offheap_memory_mb\n", + " executor_totalmem_mb = yarn_padding(executor_totalmem_mb)\n", + " if byte_to_mb(convert_to_bytes(memory_per_node)) - executor_totalmem_mb*executors_per_node > executor_totalmem_mb:\n", + " executor_memory_overhead_mb += 1024\n", + " \n", + " print('''\n", + " executors per node: {:d}\n", + " parallelism: {:d}\n", + " executor memory: {:d}m\n", + " offheap memory: {:d}m\n", + " '''.format(executors_per_node, parallelism, executor_memory_mb, offheap_memory_mb))\n", + "\n", + " conf = SparkConf() \\\n", + " .set('spark.app.name', app_name)\\\n", + " .set('spark.master',master)\\\n", + " .set('spark.executor.memory', '{:d}m'.format(executor_memory_mb))\\\n", + " .set('spark.memory.offHeap.enabled', enable_offheap)\\\n", + " .set('spark.memory.offHeap.size','{:d}m'.format(offheap_memory_mb))\\\n", + " .set('spark.sql.shuffle.partitions', parallelism)\\\n", + " .set('spark.executor.instances', '{:d}'.format(num_executors))\\\n", + " .set('spark.executor.cores','{:d}'.format(cores_per_executor))\\\n", + " .set('spark.task.cpus','{:d}'.format(1))\\\n", + " .set('spark.driver.memory', '{:d}m'.format(driver_memory_mb))\\\n", + " .set('spark.executor.memoryOverhead', '{:d}m'.format(executor_memory_overhead_mb))\\\n", + " .set('spark.driver.maxResultSize', '4g')\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " f'-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile={home}/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.driver.extraClassPath', extra_jars) \\\n", + " .set('spark.executor.extraClassPath', extra_jars) \\\n", + " .set('spark.executorEnv.PYTHONPATH',f\"{os.environ['SPARK_HOME']}python:{get_py4jzip()}\") \\\n", + " .set(\"spark.repl.class.outputDir\", output_dir) \\\n", + " .set(\"spark.sql.broadcastTimeout\", \"4800\") \\\n", + " .set('spark.serializer','org.apache.spark.serializer.KryoSerializer')\\\n", + " .set('spark.kryoserializer.buffer.max','512m')\\\n", + " .set('spark.kryo.unsafe',False)\\\n", + " .set('spark.sql.adaptive.enabled',True)\\\n", + " .set('spark.sql.autoBroadcastJoinThreshold',\"10m\")\\\n", + " .set('spark.sql.catalogImplementation','hive')\\\n", + " .set('spark.sql.optimizer.dynamicPartitionPruning.enabled',True)\\\n", + " .set('spark.cleaner.periodicGC.interval', '10s')\n", + "\n", + " return conf\n", + "\n", + "\n", + "def create_cntx_with_config(conf,conf_overwrite=None):\n", + "\n", + " importlib.reload(pyspark.java_gateway)\n", + "\n", + " def Popen(*args, **kwargs):\n", + " \"\"\"Wraps subprocess.Popen to force stdout and stderr from the child process\n", + " to pipe to this process without buffering.\n", + " \"\"\"\n", + " global spark_jvm_proc\n", + " # Override these in kwargs to avoid duplicate value errors\n", + " # Set streams to unbuffered so that we read whatever bytes are available\n", + " # when ready, https://docs.python.org/3.6/library/subprocess.html#popen-constructor\n", + " kwargs['bufsize'] = 0\n", + " # Capture everything from stdout for display in the notebook\n", + " kwargs['stdout'] = subprocess.PIPE\n", + " print(\"java proc gateway popen\")\n", + " spark_jvm_proc = subprocess.Popen(*args, **kwargs)\n", + " return spark_jvm_proc\n", + " pyspark.java_gateway.Popen = Popen\n", + "\n", + " spylon_kernel.scala_interpreter.scala_intp=None\n", + " \n", + " if conf_overwrite is not None:\n", + " conf=conf_overwrite(conf)\n", + " print(\"spark.serializer: \",conf.get(\"spark.serializer\"))\n", + " print(\"master: \",conf.get(\"spark.master\"))\n", + " \n", + " sc = SparkContext(conf = conf,master=conf.get(\"spark.master\"))\n", + " sc.setLogLevel('ERROR')\n", + " \n", + " sc.addPyFile(f\"{os.environ['SPARK_HOME']}/python/lib/pyspark.zip\")\n", + " sc.addPyFile(get_py4jzip())\n", + " \n", + " spark = SQLContext(sc)\n", + " \n", + " time.sleep(30)\n", + " \n", + " for client in clients:\n", + " pids=!ssh $client \"jps | grep CoarseGrainedExecutorBackend | cut -d' ' -f1\"\n", + " print(client,\":\",len(pids),\" \",\"\\t\".join(map(str,pids)))\n", + " \n", + " spark_session = SparkSession(sc)\n", + " spark_jvm_helpers = SparkJVMHelpers(spark_session._sc)\n", + " spylon_kernel.scala_interpreter.spark_state = spylon_kernel.scala_interpreter.SparkState(spark_session, spark_jvm_helpers, spark_jvm_proc)\n", + " \n", + " print(\"appid: \",sc.applicationId)\n", + " print(\"SparkConf:\")\n", + "\n", + " df = pd.DataFrame(sc.getConf().getAll(), columns=['key', 'value'])\n", + " display(df)\n", + "\n", + " return sc, spark" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Vanilla Spark" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def vanilla_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def vanilla_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_vanilla(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=False)\n", + " conf.set(\"spark.sql.execution.arrow.maxRecordsPerBatch\",20480)\\\n", + " .set(\"spark.sql.parquet.columnarReaderBatchSize\",20480)\\\n", + " .set(\"spark.sql.inMemoryColumnarStorage.batchSize\",20480)\n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Gluten" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def gluten_tpch_conf_overwrite(conf):\n", + " return conf\n", + "\n", + "def gluten_tpcds_conf_overwrite(conf):\n", + " conf.set('spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold', '0')\\\n", + " .set('spark.sql.optimizer.runtime.bloomFilter.enabled', 'true')\\\n", + " .set('spark.gluten.sql.columnar.joinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.physicalJoinOptimizationLevel', '18')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizeEnable', 'true')\\\n", + " .set('spark.gluten.sql.columnar.logicalJoinOptimizationLevel', '19')\n", + " return conf" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx_gluten(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name='', master='yarn', conf_overwrite=None):\n", + " conf = default_conf(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, run_gluten=True)\n", + " conf.set('spark.sql.files.maxPartitionBytes', '4g')\\\n", + " .set('spark.plugins','org.apache.gluten.GlutenPlugin')\\\n", + " .set('spark.shuffle.manager','org.apache.spark.shuffle.sort.ColumnarShuffleManager')\\\n", + " .set('spark.gluten.sql.columnar.backend.lib','velox')\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize',4096)\\\n", + " .set('spark.gluten.sql.columnar.forceshuffledhashjoin',True)\\\n", + " .set('spark.executorEnv.LD_PRELOAD', findjemalloc())\\\n", + " .set('spark.gluten.sql.columnar.coalesce.batches', 'true')\n", + " \n", + " return create_cntx_with_config(conf,conf_overwrite)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "## Create Context" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "def create_cntx(run_gluten=False, workload='tpch', app_conf_overwrite=None, server='', base_dir='', nb_name='tpc_workload.ipynb', app_name=''):\n", + " table_dir=''\n", + " extra_jars = ''\n", + " is_tpch_workload=False\n", + " is_tpcds_workload=False\n", + " workload_conf_overwrite=None\n", + " create_cntx_func=None\n", + " test_tpc=None\n", + "\n", + " if workload.lower() == 'tpch':\n", + " if not app_name:\n", + " app_name = 'tpch_power'\n", + " tabledir = tpch_tabledir\n", + " is_tpch_workload=True\n", + " elif workload.lower() == 'tpcds':\n", + " if not app_name:\n", + " app_name = 'tpcds_power'\n", + " tabledir = tpcds_tabledir\n", + " is_tpcds_workload=True\n", + " else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + " lastgit=!git --git-dir {gluten_home}/.git log --format=\"%H\" -n 1\n", + " lastgit = lastgit[0]\n", + " print(f'lastgit: {lastgit}')\n", + "\n", + " nodes=len(clients)\n", + "\n", + " if run_gluten:\n", + " jars_base=f\"{home}/jars/\"+lastgit\n", + " \n", + " for target_jar in gluten_target_jar.split(\",\"):\n", + " !ls -l {target_jar}\n", + " !mkdir -p {jars_base}\n", + " !rm -rf {jars_base}/*\n", + " !cp {target_jar} {jars_base}/\n", + " if target_jar[-4:] != '.jar':\n", + " !cp -f {target_jar} {jars_base}/gluten-{lastgit}.jar\n", + "\n", + " jars=!ls -d {jars_base}/*.jar\n", + " extra_jars=\":\".join([\"file://\"+j for j in jars])\n", + " print(f'extra_jars: {extra_jars}')\n", + "\n", + " for c in clients:\n", + " if c!=localhost:\n", + " !ssh {c} \"rm -rf {jars_base}\"\n", + " !ssh {c} \"mkdir -p {jars_base}\"\n", + " !scp {jars_base}/*.jar {c}:{jars_base} >/dev/null 2>&1\n", + "\n", + " app_name = ' '.join(['gluten', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_gluten\n", + " if is_tpch_workload:\n", + " task_per_core = gluten_tpch_task_per_core\n", + " workload_conf_overwrite = gluten_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = gluten_tpcds_task_per_core\n", + " workload_conf_overwrite = gluten_tpcds_conf_overwrite\n", + " else:\n", + " app_name = ' '.join(['vanilla', app_name, lastgit[:6]])\n", + " create_cntx_func=create_cntx_vanilla\n", + " if is_tpch_workload:\n", + " task_per_core = vanilla_tpch_task_per_core\n", + " workload_conf_overwrite = vanilla_tpch_conf_overwrite\n", + " elif is_tpcds_workload:\n", + " task_per_core = vanilla_tpcds_task_per_core\n", + " workload_conf_overwrite = vanilla_tpcds_conf_overwrite\n", + " \n", + " conf_overwrite = lambda conf: app_conf_overwrite(workload_conf_overwrite(conf))\n", + " \n", + " sc, spark = create_cntx_func(executors_per_node, cores_per_executor, task_per_core, memory_per_node, extra_jars, app_name, master, conf_overwrite)\n", + " \n", + " # Pin executors to numa nodes for Gluten\n", + " if run_gluten:\n", + " pinexecutor_numa(clients)\n", + "\n", + " appid = sc.applicationId\n", + " print(\"start run: \", appid)\n", + " \n", + " if is_tpch_workload:\n", + " test_tpc = TestTPCH(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " elif is_tpcds_workload:\n", + " test_tpc = TestTPCDS(spark, tabledir, run_gluten, server, base_dir, nb_name)\n", + " \n", + " return sc, spark, appid, test_tpc" + ] + } + ], + "metadata": { + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "364.469px", + "left": "2086.8px", + "top": "150.516px", + "width": "375px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "toc-autonumbering": true, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/benchmark_velox/params.yaml.template b/tools/workload/benchmark_velox/params.yaml.template new file mode 100644 index 000000000000..1c70e428bc99 --- /dev/null +++ b/tools/workload/benchmark_velox/params.yaml.template @@ -0,0 +1,71 @@ +# Local path to gluten project. +gluten_home: /home/sparkuser/gluten + +# Local path to gluten jar. +gluten_target_jar: /home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar + +# Spark app master. +master: yarn + +# List of workers. +clients: + - localhost + +# List of block devices. +disk_dev: + - nvme1n1 + - nvme2n1 + +# List of network devices. +nic_dev: + - ens787f0 + +# Hostname or IP to server for perf analysis. Able to connect via ssh. +server: '' + +# Specify the directory on perf analysis server. Usually a codename for this run. +base_dir: emr + +# Proxy used to connect to server for perf analysis. +proxy: '' + +# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set. +analyze_perf: True + +# Select workload. Can be either 'tpch' or 'tpcds'. +workload: tpch + +# Run with gluten. If False, run vanilla Spark. +run_gluten: True + +# TPC tables +tpch_tabledir: /tpch_sf3000 +tpcds_tabledir: /tpcds_sf3000 + +# Parallelism +executors_per_node: 32 +cores_per_executor: 8 + +gluten_tpch_task_per_core: 2 +gluten_tpcds_task_per_core: 2 +vanilla_tpch_task_per_core: 4 +vanilla_tpcds_task_per_core: 4 + +# Physical memory on each worker node. +memory_per_node: 1000g + +# Offheap ratio. 0 to disable offheap for vanilla Spark. +# onheap:offheap = 1:2 +vanilla_offheap_ratio: 2.0 +# onheap:offheap = 1:7 +gluten_offheap_ratio: 7.0 + +# spark.io.compression.codec +vanilla_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codec +gluten_codec: lz4 +# spark.gluten.sql.columnar.shuffle.codecBackend +gluten_codec_backend: '' +# spark.gluten.sql.columnar.maxBatchSize +max_batch_size: 4096 + diff --git a/tools/workload/benchmark_velox/run_tpc_workload.sh b/tools/workload/benchmark_velox/run_tpc_workload.sh new file mode 100755 index 000000000000..f6de6ff0f538 --- /dev/null +++ b/tools/workload/benchmark_velox/run_tpc_workload.sh @@ -0,0 +1,86 @@ +#! /bin/bash + +# 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. + +set -eu + +PAPERMILL_ARGS=() +OUTPUT_DIR=$PWD + +while [[ $# -gt 0 ]]; do + case $1 in + --notebook) + NOTEBOOK="$2" + shift # past argument + shift # past value + ;; + --output-dir) + OUTPUT_DIR="$2" + shift # past argument + shift # past value + ;; + --output-name) + OUTPUT_NAME="$2" + shift # past argument + shift # past value + ;; + *) + PAPERMILL_ARGS+=("$1") # save positional arg + shift # past argument + ;; + esac +done + +if [ -z ${NOTEBOOK+x} ]; then + echo "Usage: $0 --notebook NOTEBOOK [--output-dir OUTPUT_DIR] [--output-name OUTPUT_NAME] [PAPERMILL_ARGS]" + exit 0 +fi + + +BASEDIR=$(dirname $0) +echo "Script location: ${BASEDIR}" + +nbname=$(basename $NOTEBOOK .ipynb) + +if [ -z ${OUTPUT_NAME+x} ]; then output_name=$nbname; else output_name=$(basename $OUTPUT_NAME .ipynb); fi + +output_dir=$(realpath $OUTPUT_DIR) +mkdir -p $output_dir + +rename_append_appid() { + output_name=$1 + orig_nb=$2 + + output_appid=`grep "appid: " $orig_nb | awk -F' ' '{print $2}' | sed 's/....$//'` + if [ -n "$output_appid" ]; + then + rename_nb=${output_dir}/${output_name}-${output_appid}.ipynb + echo "Rename notebook $orig_nb to $rename_nb" + mv $orig_nb $rename_nb + fi +} + +run() { + output_name=${output_name}-$(date +"%H%M%S") + output_nb=${output_dir}/${output_name}.ipynb + papermill --inject-output-path $NOTEBOOK \ + ${PAPERMILL_ARGS[@]} \ + $output_nb + rename_append_appid $output_name $output_nb +} + +run + diff --git a/tools/workload/benchmark_velox/tpc_workload.ipynb b/tools/workload/benchmark_velox/tpc_workload.ipynb new file mode 100644 index 000000000000..5dcb50a8a066 --- /dev/null +++ b/tools/workload/benchmark_velox/tpc_workload.ipynb @@ -0,0 +1,381 @@ +{ + "cells": [ + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# initialization" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "tags": [ + "parameters" + ] + }, + "outputs": [], + "source": [ + "# Local path to gluten project.\n", + "gluten_home='/home/sparkuser/gluten'\n", + "\n", + "# Local path to gluten jar.\n", + "gluten_target_jar='/home/sparkuser/gluten-velox-bundle-spark3.3_2.12-centos_7_x86_64-1.3.0-SNAPSHOT.jar'\n", + "\n", + "# Spark app master. e.g. 'yarn'\n", + "master='yarn'\n", + "\n", + "# List of workers.\n", + "clients=['localhost']\n", + "\n", + "# List of block devices. e.g. ['nvme1n1', 'nvme2n1']\n", + "disk_dev=[]\n", + "\n", + "# List of network devices. e.g. ['ens787f0']\n", + "nic_dev=[]\n", + "\n", + "# Hostname or IP to server for perf analysis. Able to connect via ssh.\n", + "server=''\n", + "\n", + "# Specify the directory on perf analysis server. Usually a codename for this run.\n", + "base_dir=''\n", + "\n", + "# Proxy used to connect to server for perf analysis.\n", + "proxy=''\n", + "\n", + "# Whether to upload profile to perf analysis server and run perf analysis scripts. Only takes effect if server is set.\n", + "analyze_perf=True\n", + "\n", + "# Select workload. Can be either 'tpch' or 'tpcds'.\n", + "workload='tpch'\n", + "\n", + "# Run with gluten. If False, run vanilla Spark.\n", + "run_gluten=True\n", + "\n", + "# TPC tables\n", + "tpch_tabledir=''\n", + "tpcds_tabledir=''\n", + "\n", + "# Parallelism\n", + "executors_per_node=32\n", + "cores_per_executor=8\n", + "\n", + "gluten_tpch_task_per_core=2\n", + "gluten_tpcds_task_per_core=4\n", + "vanilla_tpch_task_per_core=8\n", + "vanilla_tpcds_task_per_core=8\n", + "\n", + "# Physical memory on each worker node.\n", + "memory_per_node='1000g'\n", + "\n", + "# Offheap ratio. 0 to disable offheap for vanilla Spark.\n", + "# onheap:offheap = 1:2\n", + "vanilla_offheap_ratio=2.0\n", + "# onheap:offheap = 1:7\n", + "gluten_offheap_ratio=7.0\n", + "\n", + "# spark.io.compression.codec\n", + "vanilla_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codec\n", + "gluten_codec='lz4'\n", + "# spark.gluten.sql.columnar.shuffle.codecBackend\n", + "gluten_codec_backend=''\n", + "# spark.gluten.sql.columnar.maxBatchSize\n", + "max_batch_size=4096" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "initialize_ipynb = !realpath native_sql_initialize.ipynb\n", + "print(f\"Running notebook: {initialize_ipynb[0]}\\n\")\n", + "%run {initialize_ipynb[0]}" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "newClients = []\n", + "for l in clients:\n", + " if l == 'localhost':\n", + " newClients.append(localhost)\n", + " else:\n", + " newClients.append(l)\n", + "clients = newClients\n", + "\n", + "if server == 'localhost':\n", + " server = localhost" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "%%javascript\n", + "IPython.notebook.kernel.execute('nb_name = \"' + IPython.notebook.notebook_name + '\"')" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "deletable": false, + "editable": false, + "run_control": { + "frozen": true + } + }, + "outputs": [], + "source": [ + "nb_name=PAPERMILL_OUTPUT_PATH" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Application Level Configuration" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "tpch_workload=False\n", + "tpcds_workload=False\n", + "\n", + "if workload.lower() == 'tpch':\n", + " tpch_workload=True\n", + "elif workload.lower() == 'tpcds':\n", + " tpcds_workload=True\n", + "else:\n", + " raise ValueError(f\"Unknown workload: {workload}\")\n", + "\n", + "def gluten_conf_overwrite(conf):\n", + " conf.set('spark.gluten.sql.columnar.shuffle.codec', gluten_codec)\\\n", + " .set('spark.gluten.sql.columnar.shuffle.codecBackend', gluten_codec_backend)\\\n", + " .set('spark.gluten.sql.columnar.maxBatchSize', max_batch_size)\\\n", + " .set('spark.executor.extraJavaOptions',\\\n", + " '-XX:+UseParallelOldGC -XX:ParallelGCThreads=2 -XX:NewRatio=1 -XX:SurvivorRatio=1 -XX:+UseCompressedOops -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -XX:ErrorFile=/home/sparkuser/logs/java/hs_err_pid%p.log')\\\n", + " .set('spark.gluten.memory.overAcquiredMemoryRatio','0')\\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def vanilla_conf_overwrite(conf):\n", + " conf.set('spark.io.compression.codec', vanilla_codec)\\\n", + " .set('spark.executorEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + " .set('spark.yarn.appMasterEnv.LD_LIBRARY_PATH',f\"{os.getenv('HADOOP_HOME')}/lib/native/\") \\\n", + "\n", + " if tpch_workload:\n", + " pass\n", + " elif tpcds_workload:\n", + " pass\n", + " return conf\n", + "\n", + "def app_conf_overwrite(conf):\n", + " if run_gluten:\n", + " return gluten_conf_overwrite(conf)\n", + " return vanilla_conf_overwrite(conf)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Run Workload" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Config and clean pagecache before each run\n", + "config_pagecache(clients, run_gluten)\n", + "dropcache(clients)\n", + "print_kernel_params(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "# Create SparkSession\n", + "sc, spark, appid, test_tpc=create_cntx(run_gluten, workload, app_conf_overwrite, server, base_dir, nb_name)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if run_gluten:\n", + " config_mem_cgroup(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.start_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.power_run(explain=False, print_result=False, load_table=True)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.stop_monitor(clients)" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "if analyze_perf:\n", + " test_tpc.run_perf_analysis(disk_dev, nic_dev)" + ] + }, + { + "cell_type": "markdown", + "metadata": {}, + "source": [ + "# Show Performance" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": {}, + "outputs": [], + "source": [ + "test_tpc.print_result()" + ] + }, + { + "cell_type": "code", + "execution_count": null, + "metadata": { + "code_folding": [] + }, + "outputs": [], + "source": [ + "for client in clients:\n", + " draw_sar(appid, qtime=test_tpc.result, disk_dev=disk_dev, nic_dev=nic_dev, client=client)" + ] + } + ], + "metadata": { + "celltoolbar": "Tags", + "hide_input": false, + "kernelspec": { + "display_name": "Python 3 (ipykernel)", + "language": "python", + "name": "python3" + }, + "language_info": { + "codemirror_mode": { + "name": "ipython", + "version": 3 + }, + "file_extension": ".py", + "mimetype": "text/x-python", + "name": "python", + "nbconvert_exporter": "python", + "pygments_lexer": "ipython3", + "version": "3.10.12" + }, + "nbTranslate": { + "displayLangs": [ + "*" + ], + "hotkey": "alt-t", + "langInMainMenu": true, + "sourceLang": "en", + "targetLang": "fr", + "useGoogleTranslate": true + }, + "toc": { + "base_numbering": 1, + "nav_menu": {}, + "number_sections": true, + "sideBar": false, + "skip_h1_title": false, + "title_cell": "Table of Contents", + "title_sidebar": "Contents", + "toc_cell": false, + "toc_position": { + "height": "428.672px", + "left": "1339.91px", + "top": "374.297px", + "width": "456.969px" + }, + "toc_section_display": true, + "toc_window_display": true + }, + "varInspector": { + "cols": { + "lenName": 16, + "lenType": 16, + "lenVar": 40 + }, + "kernels_config": { + "python": { + "delete_cmd_postfix": "", + "delete_cmd_prefix": "del ", + "library": "var_list.py", + "varRefreshCmd": "print(var_dic_list())" + }, + "r": { + "delete_cmd_postfix": ") ", + "delete_cmd_prefix": "rm(", + "library": "var_list.r", + "varRefreshCmd": "cat(var_dic_list()) " + } + }, + "types_to_exclude": [ + "module", + "function", + "builtin_function_or_method", + "instance", + "_Feature" + ], + "window_display": false + } + }, + "nbformat": 4, + "nbformat_minor": 4 +} diff --git a/tools/workload/tpcds/README.md b/tools/workload/tpcds/README.md index efe3083d0cfe..8091054a0fb5 100644 --- a/tools/workload/tpcds/README.md +++ b/tools/workload/tpcds/README.md @@ -29,5 +29,5 @@ val dbgenDir = "/PATH/TO/TPCDS_DBGEN" // location of dbgen Currently, Gluten with Velox can support Parquet file format and three compression codec including snappy, gzip, zstd. ## Test Queries -We provide the test queries in [TPC-DS Queries](../../../gluten-core/src/test/resources/tpcds-queries). +We provide the test queries in [TPC-DS Queries](../../../tools/gluten-it/common/src/main/resources/tpcds-queries). We also provide a Scala script in [Run TPC-DS](./run_tpcds) directory about how to run TPC-DS queries. diff --git a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh index 8f62560dc168..60f10f094d5b 100644 --- a/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh +++ b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -22,14 +21,13 @@ cat tpcds_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --executor-memory 25g \ --executor-cores 8 \ --master yarn \ - --driver-memory 50g \ + --driver-memory 10g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar} diff --git a/tools/workload/tpch/README.md b/tools/workload/tpch/README.md index 65a7deb09f99..4180df60f8b7 100644 --- a/tools/workload/tpch/README.md +++ b/tools/workload/tpch/README.md @@ -48,6 +48,6 @@ val dwrf_file_path = "/PATH/TO/TPCH_DWRF_PATH" ``` ## Test Queries -We provide the test queries in [TPC-H queries](../../../gluten-core/src/test/resources/tpch-queries). +We provide the test queries in [TPC-H queries](../../../tools/gluten-it/common/src/main/resources/tpch-queries). We also provide a scala script in [Run TPC-H](./run_tpch/) directory about how to run TPC-H queries. Please note if you are using DWRF test, please remember to set the file format to DWRF in the code. diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala deleted file mode 100644 index 958a98f57020..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala +++ /dev/null @@ -1,62 +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. - */ -import org.apache.spark.sql.execution.debug._ -import scala.io.Source -import java.io.File -import java.util.Arrays -import org.apache.spark.sql.types.{DoubleType, TimestampType, LongType, IntegerType} - -val parquet_file_path = "/PATH/TO/TPCH_PARQUET_PATH" -val dwrf_file_path = "/PATH/TO/TPCH_DWRF_PATH" - -val lineitem_parquet_path = "file://" + parquet_file_path + "/lineitem" -val customer_parquet_path = "file://" + parquet_file_path + "/customer" -val nation_parquet_path = "file://" + parquet_file_path + "/nation" -val orders_parquet_path = "file://" + parquet_file_path + "/orders" -val part_parquet_path = "file://" + parquet_file_path + "/part" -val partsupp_parquet_path = "file://" + parquet_file_path + "/partsupp" -val region_path_path = "file://" + parquet_file_path + "/region" -val supplier_parquet_path = "file://" + parquet_file_path + "/supplier" - -val lineitem = spark.read.format("parquet").load(lineitem_parquet_path) -val customer = spark.read.format("parquet").load(customer_parquet_path) -val nation = spark.read.format("parquet").load(nation_parquet_path) -val orders = spark.read.format("parquet").load(orders_parquet_path) -val part = spark.read.format("parquet").load(part_parquet_path) -val partsupp = spark.read.format("parquet").load(partsupp_parquet_path) -val region = spark.read.format("parquet").load(region_path_path) -val supplier = spark.read.format("parquet").load(supplier_parquet_path) - -val lineitem_dwrf_path = "file://" + dwrf_file_path + "/lineitem" -val customer_dwrf_path = "file://" + dwrf_file_path + "/customer" -val nation_dwrf_path = "file://" + dwrf_file_path + "/nation" -val orders_dwrf_path = "file://" + dwrf_file_path + "/orders" -val part_dwrf_path = "file://" + dwrf_file_path + "/part" -val partsupp_dwrf_path = "file://" + dwrf_file_path + "/partsupp" -val region_dwrf_path = "file://" + dwrf_file_path + "/region" -val supplier_dwrf_path = "file://" + dwrf_file_path + "/supplier" - -lineitem.write.mode("append").format("dwrf").save(lineitem_dwrf_path) -customer.write.mode("append").format("dwrf").save(customer_dwrf_path) -nation.write.mode("append").format("dwrf").save(nation_dwrf_path) -orders.write.mode("append").format("dwrf").save(orders_dwrf_path) -part.write.mode("append").format("dwrf").save(part_dwrf_path) -partsupp.write.mode("append").format("dwrf").save(partsupp_dwrf_path) -region.write.mode("append").format("dwrf").save(region_dwrf_path) -supplier.write.mode("append").format("dwrf").save(supplier_dwrf_path) - - diff --git a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh deleted file mode 100644 index 2dc05c9aa7dd..000000000000 --- a/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh +++ /dev/null @@ -1,47 +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. -batchsize=20480 - -export GLUTEN_HOME=/PATH/TO/gluten/ -#please choose right os system jar -export GLUTEN_JVM_JAR=${GLUTEN_HOME}/package/target/ -SPARK_HOME=/home/sparkuser/spark/ - -cat tpch_convert_parquet_dwrf.scala | ${SPARK_HOME}/bin/spark-shell \ - --name convert_parquet_dwrf \ - --master yarn \ - --deploy-mode client \ - --driver-memory 20g \ - --executor-cores 8 \ - --num-executors 14 \ - --executor-memory 30g \ - --conf spark.plugins=org.apache.gluten.GlutenPlugin \ - --conf spark.driver.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.executor.extraClassPath=${GLUTEN_JVM_JAR} \ - --conf spark.memory.offHeap.size=30g \ - --conf spark.executor.memoryOverhead=5g \ - --conf spark.driver.maxResultSize=32g \ - --conf spark.sql.autoBroadcastJoinThreshold=-1 \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ - --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ - --conf spark.sql.broadcastTimeout=4800 \ - --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.adaptive.enabled=true \ - --conf spark.sql.shuffle.partitions=112 \ - --conf spark.sql.sources.useV1SourceList=avro \ - --conf spark.shuffle.manager=org.apache.spark.shuffle.sort.ColumnarShuffleManager \ - --conf spark.sql.files.maxPartitionBytes=1073741824 \ diff --git a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh index 29512ed80b15..8db9d443331d 100644 --- a/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh +++ b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -batchsize=10240 SPARK_HOME=/home/sparkuser/spark/ spark_sql_perf_jar=/PATH/TO/spark-sql-perf_2.12-0.5.1-SNAPSHOT.jar cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ @@ -25,11 +24,10 @@ cat tpch_datagen_parquet.scala | ${SPARK_HOME}/bin/spark-shell \ --driver-memory 50g \ --deploy-mode client \ --conf spark.executor.memoryOverhead=1g \ - --conf spark.sql.parquet.columnarReaderBatchSize=${batchsize} \ - --conf spark.sql.inMemoryColumnarStorage.batchSize=${batchsize} \ - --conf spark.sql.execution.arrow.maxRecordsPerBatch=${batchsize} \ --conf spark.sql.broadcastTimeout=4800 \ --conf spark.driver.maxResultSize=4g \ - --conf spark.sql.sources.useV1SourceList=avro \ --conf spark.sql.shuffle.partitions=224 \ + --conf spark.sql.parquet.compression.codec=snappy \ + --conf spark.network.timeout=800s \ + --conf spark.executor.heartbeatInterval=200s \ --jars ${spark_sql_perf_jar}