diff --git a/.github/workflows/ch_code_style.yml b/.github/workflows/ch_code_style.yml index 0ed3e9b61e6e..1428bdc166be 100644 --- a/.github/workflows/ch_code_style.yml +++ b/.github/workflows/ch_code_style.yml @@ -32,7 +32,7 @@ jobs: - check: 'cpp-ch' exclude: '' steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Run clang-format style check for C/C++ programs. uses: jidicula/clang-format-action@v4.11.0 with: diff --git a/.github/workflows/clickhouse_be_trigger.yml b/.github/workflows/clickhouse_be_trigger.yml index ba5dbeb71a7c..6825dd91acc4 100644 --- a/.github/workflows/clickhouse_be_trigger.yml +++ b/.github/workflows/clickhouse_be_trigger.yml @@ -35,7 +35,7 @@ jobs: runs-on: ubuntu-latest steps: - name: Checkout code - uses: actions/checkout@v2 + uses: actions/checkout@v4 - name: Sleep for Dev PR workflow done run: | sleep 15 diff --git a/.github/workflows/code_style.yml b/.github/workflows/code_style.yml index 612b04ee3af8..e932790755e6 100644 --- a/.github/workflows/code_style.yml +++ b/.github/workflows/code_style.yml @@ -34,7 +34,7 @@ jobs: - check: 'cpp/velox' exclude: '' steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Run clang-format style check for C/C++ programs. uses: jidicula/clang-format-action@v3.5.1 with: diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml new file mode 100644 index 000000000000..265fd6e082bd --- /dev/null +++ b/.github/workflows/stale.yml @@ -0,0 +1,31 @@ +# 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. + +name: 'Close stale Rs' +on: + schedule: + - cron: '30 1 * * *' + +jobs: + stale: + runs-on: ubuntu-latest + steps: + - uses: actions/stale@v8 + with: + stale-pr-message: 'This PR is stale because it has been open 45 days with no activity. Remove stale label or comment or this will be closed in 10 days.' + close-pr-message: 'This PR was closed because it has been stalled for 10 days with no activity.' + days-before-pr-stale: 45 + days-before-pr-close: 10 + operations-per-run: 300 diff --git a/.github/workflows/velox_be.yml b/.github/workflows/velox_be.yml index c41e2eb094c8..3ebf43e48564 100644 --- a/.github/workflows/velox_be.yml +++ b/.github/workflows/velox_be.yml @@ -28,7 +28,6 @@ on: - 'shims/**' - 'tools/gluten-it/**' - 'tools/gluten-te/**' - - 'ep/build-arrow/**' - 'ep/build-velox/**' - 'cpp/*' - 'cpp/CMake/**' @@ -46,7 +45,7 @@ jobs: ubuntu2004-test-spark32: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -55,17 +54,14 @@ jobs: - name: Build Gluten velox third party run: | docker exec ubuntu2004-test-$GITHUB_RUN_ID bash -c ' - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ cd /opt/gluten/ep/build-velox/src && \ ./get_velox.sh --velox_home=/opt/velox && \ - ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' + ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON --build_tests=ON' - name: Build Gluten CPP library run: | docker exec ubuntu2004-test-$GITHUB_RUN_ID bash -c ' cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox --build_tests=ON --build_examples=ON --build_benchmarks=ON' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox --build_tests=ON --build_examples=ON --build_benchmarks=ON' - name: Run CPP unit test run: | docker exec ubuntu2004-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/cpp/build && \ @@ -95,7 +91,7 @@ jobs: ubuntu2004-test-slow-spark32: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -104,9 +100,6 @@ jobs: - name: Build Gluten velox third party run: | docker exec ubuntu2004-test-slow-$GITHUB_RUN_ID bash -c ' - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ cd /opt/gluten/ep/build-velox/src && \ ./get_velox.sh --velox_home=/opt/velox && \ ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' @@ -114,7 +107,7 @@ jobs: run: | docker exec ubuntu2004-test-slow-$GITHUB_RUN_ID bash -c ' cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox' - name: Build and run unit test for Spark 3.2.2(slow tests) run: | docker exec ubuntu2004-test-slow-$GITHUB_RUN_ID bash -c ' @@ -136,7 +129,7 @@ jobs: ubuntu2004-test-spark33-slow: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -145,9 +138,6 @@ jobs: - name: Build Gluten velox third party run: | docker exec ubuntu2004-test-spark33-slow-$GITHUB_RUN_ID bash -l -c ' - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ cd /opt/gluten/ep/build-velox/src && \ ./get_velox.sh --velox_home=/opt/velox && \ ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' @@ -155,7 +145,7 @@ jobs: run: | docker exec ubuntu2004-test-spark33-slow-$GITHUB_RUN_ID bash -l -c ' cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox' - name: Build and Run unit test for Spark 3.3.1(slow tests) run: | docker exec ubuntu2004-test-spark33-slow-$GITHUB_RUN_ID bash -l -c 'cd /opt/gluten && \ @@ -176,7 +166,7 @@ jobs: ubuntu2004-test-spark33: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -185,9 +175,6 @@ jobs: - name: Build Gluten velox third party run: | docker exec ubuntu2004-test-spark33-$GITHUB_RUN_ID bash -c ' - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ cd /opt/gluten/ep/build-velox/src && \ ./get_velox.sh --velox_home=/opt/velox && \ ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' @@ -195,7 +182,7 @@ jobs: run: | docker exec ubuntu2004-test-spark33-$GITHUB_RUN_ID bash -c ' cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox --build_examples=ON' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox --build_examples=ON' - name: Build and Run unit test for Spark 3.3.1(other tests) run: | docker exec ubuntu2004-test-spark33-$GITHUB_RUN_ID bash -c 'cd /opt/gluten && \ @@ -205,11 +192,78 @@ jobs: if: ${{ always() }} run: | docker stop ubuntu2004-test-spark33-$GITHUB_RUN_ID || true + + ubuntu2004-test-spark34-slow: + runs-on: velox-self-hosted + steps: + - uses: actions/checkout@v4 + - name: Setup docker container + run: | + docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ + -v $PWD:/opt/gluten --name ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID -e NUM_THREADS=30 -detach 10.0.2.4:5000/gluten-dev/ubuntu:20.04 \ + 'cd /opt/gluten && sleep 14400' + - name: Build Gluten velox third party + run: | + docker exec ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID bash -l -c ' + cd /opt/gluten/ep/build-velox/src && \ + ./get_velox.sh --velox_home=/opt/velox && \ + ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' + - name: Build Gluten CPP library + run: | + docker exec ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID bash -l -c ' + cd /opt/gluten/cpp && \ + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox ' + - name: Build and Run unit test for Spark 3.4.1(slow tests) + run: | + docker exec ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID bash -l -c 'cd /opt/gluten && \ + mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark331" -DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest' + - name: TPC-H SF1.0 && TPC-DS SF1.0 Parquet local spark3.4 + run: | + docker exec ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID bash -l -c 'cd /opt/gluten/tools/gluten-it && \ + mvn clean install -Pspark-3.4 \ + && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --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' + - name: Exit docker container + if: ${{ always() }} + run: | + docker stop ubuntu2004-test-spark34-slow-$GITHUB_RUN_ID || true + + ubuntu2004-test-spark34: + runs-on: velox-self-hosted + steps: + - uses: actions/checkout@v4 + - name: Setup docker container + run: | + docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ + -v $PWD:/opt/gluten --name ubuntu2004-test-spark34-$GITHUB_RUN_ID -e NUM_THREADS=30 -detach 10.0.2.4:5000/gluten-dev/ubuntu:20.04 \ + 'cd /opt/gluten && sleep 14400' + - name: Build Gluten velox third party + run: | + docker exec ubuntu2004-test-spark34-$GITHUB_RUN_ID bash -c ' + cd /opt/gluten/ep/build-velox/src && \ + ./get_velox.sh --velox_home=/opt/velox && \ + ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' + - name: Build Gluten CPP library + run: | + docker exec ubuntu2004-test-spark34-$GITHUB_RUN_ID bash -c ' + cd /opt/gluten/cpp && \ + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox --build_examples=ON' + - name: Build and Run unit test for Spark 3.4.1(other tests) + run: | + docker exec ubuntu2004-test-spark34-$GITHUB_RUN_ID bash -c 'cd /opt/gluten && \ + mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -Pspark-ut -DargLine="-Dspark.test.home=/opt/spark331" -DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,io.glutenproject.tags.UDFTest,io.glutenproject.tags.SkipTestTags && \ + mvn test -Pspark-3.4 -Pbackends-velox -DtagsToExclude=None -DtagsToInclude=io.glutenproject.tags.UDFTest' + - name: Exit docker container + if: ${{ always() }} + run: | + docker stop ubuntu2004-test-spark34-$GITHUB_RUN_ID || true ubuntu2204-test: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -218,17 +272,14 @@ jobs: - name: Build Gluten velox third party run: | docker exec ubuntu2204-test-$GITHUB_RUN_ID bash -c ' - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON && \ - ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON --enable_hdfs=ON --enable_s3=ON' + ./get_velox.sh --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON && \ + ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON' - name: Build Gluten CPP library run: | docker exec ubuntu2204-test-$GITHUB_RUN_ID bash -c ' cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON' - name: Build for Spark 3.2.2 run: | docker exec ubuntu2204-test-$GITHUB_RUN_ID bash -c ' @@ -269,6 +320,19 @@ jobs: --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ && GLUTEN_IT_JVM_ARGS=-Xmx20G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=30g -s=10.0 --threads=32 --iterations=1' + - name: Build for Spark 3.4.1 + run: | + docker exec ubuntu2204-test-$GITHUB_RUN_ID bash -c ' + cd /opt/gluten && \ + mvn clean install -Pspark-3.4 -Pbackends-velox -Prss -DskipTests' + - name: TPC-H SF1.0 && TPC-DS SF10.0 Parquet local spark3.4 + run: | + docker exec ubuntu2204-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ + mvn clean install -Pspark-3.4 \ + && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ + && GLUTEN_IT_JVM_ARGS=-Xmx20G sbin/gluten-it.sh queries-compare \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=30g -s=10.0 --threads=32 --iterations=1' - name: Exit docker container if: ${{ always() }} run: | @@ -277,7 +341,7 @@ jobs: centos8-test: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -287,18 +351,16 @@ jobs: run: | docker exec centos8-test-$GITHUB_RUN_ID bash -c ' source /env.sh && \ - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ + sudo yum -y install patch && \ cd /opt/gluten/ep/build-velox/src && \ - ./get_velox.sh --velox_home=/opt/velox && \ - ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON' + ./get_velox.sh --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON && \ + ./build_velox.sh --velox_home=/opt/velox --enable_ep_cache=ON --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON' - name: Build Gluten CPP library run: | docker exec centos8-test-$GITHUB_RUN_ID bash -c ' source /env.sh && \ cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox --enable_hdfs=ON --enable_s3=ON --enable_gcs=ON' - name: Build for Spark 3.2.2 run: | docker exec centos8-test-$GITHUB_RUN_ID bash -c ' @@ -312,14 +374,14 @@ jobs: --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1' - - name: (To be fixed) TPC-H SF1.0 && TPC-DS SF30.0 Parquet local spark3.2 random kill tasks + - name: TPC-H SF1.0 && TPC-DS SF30.0 Parquet local spark3.2 random kill tasks run: | docker exec centos8-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ mvn clean install -Pspark-3.2 \ && GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries \ --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 --skip-data-gen --random-kill-tasks \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1 --skip-data-gen --random-kill-tasks' || true + --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1 --skip-data-gen --random-kill-tasks' - name: Exit docker container if: ${{ always() }} run: | @@ -328,7 +390,7 @@ jobs: centos7-test: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -338,9 +400,7 @@ jobs: run: | docker exec centos7-test-$GITHUB_RUN_ID bash -c ' source /env.sh && \ - cd /opt/gluten/ep/build-arrow/src && \ - ./get_arrow.sh --arrow_home=/opt/arrow && \ - ./build_arrow.sh --arrow_home=/opt/arrow --enable_ep_cache=ON && \ + sudo yum -y install patch && \ cd /opt/gluten/ep/build-velox/src && \ ./get_velox.sh --velox_home=/opt/velox && \ ./build_velox.sh --run_setup_script=OFF --velox_home=/opt/velox --enable_ep_cache=ON' @@ -349,7 +409,7 @@ jobs: docker exec centos7-test-$GITHUB_RUN_ID bash -c ' source /env.sh && \ cd /opt/gluten/cpp && \ - ./compile.sh --build_velox_backend=ON --arrow_home=/opt/arrow --velox_home=/opt/velox' + ./compile.sh --build_velox_backend=ON --velox_home=/opt/velox' - name: Build for Spark 3.2.2 run: | docker exec centos7-test-$GITHUB_RUN_ID bash -c ' @@ -363,19 +423,52 @@ jobs: --local --preset=velox --benchmark-type=h --error-on-memleak --disable-aqe --off-heap-size=10g -s=1.0 --threads=16 --iterations=1 \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh queries-compare \ --local --preset=velox --benchmark-type=ds --error-on-memleak --off-heap-size=50g -s=30.0 --threads=32 --iterations=1' - - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory + - name: TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory, memory isolation off run: | docker exec centos7-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ mvn clean install -Pspark-3.2 \ && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ - --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=8 --shuffle-partitions=72 --iterations=1 \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ --skip-data-gen -m=OffHeapExecutionMemory \ -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ + -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ + -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ + -d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \ + -d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5' + - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q67/Q95 low memory, memory isolation on + run: | + docker exec centos7-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ + mvn clean install -Pspark-3.2 \ + && GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q67,q95 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ + --skip-data-gen -m=OffHeapExecutionMemory \ -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ -d=OFFHEAP_SIZE:5g,spark.memory.offHeap.size=5g \ -d=OFFHEAP_SIZE:3g,spark.memory.offHeap.size=3g \ -d=OVER_ACQUIRE:0.3,spark.gluten.memory.overAcquiredMemoryRatio=0.3 \ -d=OVER_ACQUIRE:0.5,spark.gluten.memory.overAcquiredMemoryRatio=0.5' || true + - name: TPC-DS SF30.0 Parquet local spark3.2 Q23A/Q23B low memory + run: | + docker exec centos7-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ + GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q23a,q23b -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ + --skip-data-gen -m=OffHeapExecutionMemory \ + -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ + -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ + -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ + -d=PARTIAL_MODE:ABANDONED,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ + -d=PARTIAL_MODE:CACHED,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=1.0,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0 \ + -d=PARTIAL_MODE:FLUSHED,spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio=0.05,spark.gluten.sql.columnar.backend.velox.maxExtendedPartialAggregationMemoryRatio=0.1,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct=100,spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows=0' + - name: (To be fixed) TPC-DS SF30.0 Parquet local spark3.2 Q97 low memory + run: | + docker exec centos7-test-$GITHUB_RUN_ID bash -c 'cd /opt/gluten/tools/gluten-it && \ + GLUTEN_IT_JVM_ARGS=-Xmx50G sbin/gluten-it.sh parameterized \ + --local --preset=velox --benchmark-type=ds --error-on-memleak --queries=q97 -s=30.0 --threads=12 --shuffle-partitions=72 --iterations=1 \ + --skip-data-gen -m=OffHeapExecutionMemory \ + -d=ISOLATION:OFF,spark.gluten.memory.isolation=false \ + -d=ISOLATION:ON,spark.gluten.memory.isolation=true,spark.memory.storageFraction=0.1 \ + -d=OFFHEAP_SIZE:2g,spark.memory.offHeap.size=2g \ + -d=OFFHEAP_SIZE:1g,spark.memory.offHeap.size=1g' || true - name: Exit docker container if: ${{ always() }} run: | @@ -384,7 +477,7 @@ jobs: static-build-test: runs-on: velox-self-hosted steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - name: Setup docker container run: | docker run --rm --init --privileged --ulimit nofile=65536:65536 --ulimit core=-1 --security-opt seccomp=unconfined \ @@ -394,10 +487,11 @@ jobs: run: | docker exec -i static-build-test-$GITHUB_RUN_ID bash -c ' source /env.sh && \ + sudo yum -y install patch && \ cd /opt/gluten && \ sudo -E ./dev/vcpkg/setup-build-depends.sh && \ source ./dev/vcpkg/env.sh && \ - ./dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON' + ./dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_gcs=ON --enable_hdfs=ON' - name: Build for Spark 3.2.2 run: | docker exec static-build-test-$GITHUB_RUN_ID bash -c ' diff --git a/.github/workflows/velox_tpch_bench.yml b/.github/workflows/velox_tpch_bench.yml index 98d06c72142f..5186f9ae863d 100644 --- a/.github/workflows/velox_tpch_bench.yml +++ b/.github/workflows/velox_tpch_bench.yml @@ -26,7 +26,7 @@ jobs: runs-on: velox-self-hosted container: ubuntu:22.04 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - run: apt-get update && apt-get install ca-certificates -y && update-ca-certificates - run: sed -i 's/http\:\/\/archive.ubuntu.com/https\:\/\/mirrors.ustc.edu.cn/g' /etc/apt/sources.list - run: apt-get update diff --git a/.github/workflows/velox_tpch_merge.yml b/.github/workflows/velox_tpch_merge.yml index 451688c55eaf..f323e45382b9 100644 --- a/.github/workflows/velox_tpch_merge.yml +++ b/.github/workflows/velox_tpch_merge.yml @@ -31,7 +31,6 @@ on: - 'shims/**' - 'tools/gluten-it/**' - 'tools/gluten-te/**' - - 'ep/build-arrow/**' - 'ep/build-velox/**' - 'cpp/*' - 'cpp/CMake/**' @@ -45,7 +44,7 @@ jobs: runs-on: velox-self-hosted container: ubuntu:22.04 steps: - - uses: actions/checkout@v2 + - uses: actions/checkout@v4 - run: apt-get update && apt-get install ca-certificates -y && update-ca-certificates - run: sed -i 's/http\:\/\/archive.ubuntu.com/https\:\/\/mirrors.ustc.edu.cn/g' /etc/apt/sources.list - run: apt-get update diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index 7a932b3589fe..fa1989758f7d 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -53,7 +53,7 @@ org.apache.spark - spark-hive-thriftserver_${scala.binary.version} + spark-hive_${scala.binary.version} provided diff --git a/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java b/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java index a5eaa0de1ee5..a069d2df491b 100644 --- a/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/execution/ColumnarNativeIterator.java @@ -29,6 +29,8 @@ public ColumnarNativeIterator(Iterator delegated) { super(delegated); } + private transient ColumnarBatch nextBatch = null; + private static byte[] longtoBytes(long data) { return new byte[] { (byte) ((data >> 56) & 0xff), @@ -42,6 +44,22 @@ private static byte[] longtoBytes(long data) { }; } + @Override + public boolean hasNext() { + while (delegated.hasNext()) { + nextBatch = delegated.next(); + if (nextBatch.numRows() > 0) { + return true; + } + } + return false; + } + + @Override + public ColumnarBatch nextColumnarBatch() { + return nextBatch; + } + @Override public byte[] next() { ColumnarBatch nextBatch = nextColumnarBatch(); diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BlockSplitIterator.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BlockSplitIterator.java index facb9f5eed20..3c0d919f6f5e 100644 --- a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BlockSplitIterator.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/BlockSplitIterator.java @@ -33,7 +33,8 @@ public BlockSplitIterator(Iterator in, IteratorOptions options) { options.getExpr(), options.getRequiredFields(), options.getPartitionNum(), - options.getBufferSize()); + options.getBufferSize(), + options.getHashAlgorithm()); } private native long nativeCreate( @@ -42,7 +43,8 @@ private native long nativeCreate( String expr, String schema, int partitionNum, - int bufferSize); + int bufferSize, + String hashAlgorithm); private native void nativeClose(long instance); @@ -80,6 +82,8 @@ public static class IteratorOptions implements Serializable { private String expr; private String requiredFields; + private String hashAlgorithm; + public int getPartitionNum() { return partitionNum; } @@ -119,5 +123,13 @@ public String getRequiredFields() { public void setRequiredFields(String requiredFields) { this.requiredFields = requiredFields; } + + public String getHashAlgorithm() { + return hashAlgorithm; + } + + public void setHashAlgorithm(String hashAlgorithm) { + this.hashAlgorithm = hashAlgorithm; + } } } diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeExpressionEvaluator.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeExpressionEvaluator.java index 0d598cf6cba1..3322afe0b9d5 100644 --- a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeExpressionEvaluator.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHNativeExpressionEvaluator.java @@ -27,7 +27,6 @@ import io.glutenproject.substrait.plan.PlanNode; import com.google.protobuf.Any; -import io.substrait.proto.Plan; import org.apache.spark.SparkConf; import org.apache.spark.sql.internal.SQLConf; @@ -80,12 +79,12 @@ private PlanNode buildNativeConfNode(Map confs) { // Used by WholeStageTransform to create the native computing pipeline and // return a columnar result iterator. public GeneralOutIterator createKernelWithBatchIterator( - Plan wsPlan, List iterList, boolean materializeInput) { + byte[] wsPlan, List iterList, boolean materializeInput) { long allocId = CHNativeMemoryAllocators.contextInstance().getNativeInstanceId(); long handle = jniWrapper.nativeCreateKernelWithIterator( allocId, - getPlanBytesBuf(wsPlan), + wsPlan, iterList.toArray(new GeneralInIterator[0]), buildNativeConfNode( GlutenConfig.getNativeBackendConf( @@ -115,10 +114,6 @@ public GeneralOutIterator createKernelWithBatchIterator( return createOutIterator(handle); } - private byte[] getPlanBytesBuf(Plan planNode) { - return planNode.toByteArray(); - } - private GeneralOutIterator createOutIterator(long nativeHandle) { return new BatchIterator(nativeHandle); } diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java index bd40b8fef5fc..ec773343a09c 100644 --- a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/CHShuffleSplitterJniWrapper.java @@ -31,7 +31,8 @@ public long make( String localDirs, int subDirsPerLocalDir, boolean preferSpill, - long spillThreshold) { + long spillThreshold, + String hashAlgorithm) { return nativeMake( part.getShortName(), part.getNumPartitions(), @@ -45,7 +46,8 @@ public long make( localDirs, subDirsPerLocalDir, preferSpill, - spillThreshold); + spillThreshold, + hashAlgorithm); } public long makeForRSS( @@ -55,6 +57,7 @@ public long makeForRSS( int bufferSize, String codec, long spillThreshold, + String hashAlgorithm, Object pusher) { return nativeMakeForRSS( part.getShortName(), @@ -66,6 +69,7 @@ public long makeForRSS( bufferSize, codec, spillThreshold, + hashAlgorithm, pusher); } @@ -82,7 +86,8 @@ public native long nativeMake( String localDirs, int subDirsPerLocalDir, boolean preferSpill, - long spillThreshold); + long spillThreshold, + String hashAlgorithm); public native long nativeMakeForRSS( String shortName, @@ -94,6 +99,7 @@ public native long nativeMakeForRSS( int bufferSize, String codec, long spillThreshold, + String hashAlgorithm, Object pusher); public native void split(long splitterId, long block); diff --git a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/StorageJoinBuilder.java b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/StorageJoinBuilder.java index 333889939a00..7bbc3ef528ea 100644 --- a/backends-clickhouse/src/main/java/io/glutenproject/vectorized/StorageJoinBuilder.java +++ b/backends-clickhouse/src/main/java/io/glutenproject/vectorized/StorageJoinBuilder.java @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.storage.CHShuffleReadStreamFactory; -import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; @@ -88,8 +87,8 @@ public static long build( /** create table named struct */ private static NamedStruct toNameStruct(List output) { - ArrayList typeList = ConverterUtils.collectAttributeTypeNodes(output); - ArrayList nameList = ConverterUtils.collectAttributeNamesWithExprId(output); + List typeList = ConverterUtils.collectAttributeTypeNodes(output); + List nameList = ConverterUtils.collectAttributeNamesWithExprId(output); Type.Struct.Builder structBuilder = Type.Struct.newBuilder(); for (TypeNode typeNode : typeList) { structBuilder.addTypes(typeNode.toProtobuf()); diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHBackend.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHBackend.scala index b7774575c1d1..ad65c65e64e0 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHBackend.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHBackend.scala @@ -20,11 +20,11 @@ import io.glutenproject.{CH_BRANCH, CH_COMMIT, GlutenConfig, GlutenPlugin} import io.glutenproject.backendsapi._ import io.glutenproject.expression.WindowFunctionsBuilder import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat -import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat.{JsonReadFormat, MergeTreeReadFormat, OrcReadFormat, ParquetReadFormat, TextReadFormat} +import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat._ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, DenseRank, Lag, Lead, NamedExpression, Rank, RowNumber} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, DenseRank, Lag, Lead, NamedExpression, Rank, RowNumber} import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning} import org.apache.spark.sql.execution.SparkPlan @@ -90,6 +90,24 @@ object CHBackendSettings extends BackendSettingsApi with Logging { private val GLUTEN_CLICKHOUSE_SHUFFLE_SUPPORTED_CODEC: Set[String] = Set("lz4", "zstd", "snappy") + // The algorithm for hash partition of the shuffle + private val GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM: String = + GlutenConfig.GLUTEN_CONFIG_PREFIX + CHBackend.BACKEND_NAME + + ".shuffle.hash.algorithm" + // valid values are: cityHash64 or sparkMurmurHash3_32 + private val GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM_DEFAULT = "cityHash64" + def shuffleHashAlgorithm: String = { + val algorithm = SparkEnv.get.conf.get( + CHBackendSettings.GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM, + CHBackendSettings.GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM_DEFAULT + ) + if (!algorithm.equals("cityHash64") && !algorithm.equals("sparkMurmurHash3_32")) { + CHBackendSettings.GLUTEN_CLICKHOUSE_SHUFFLE_HASH_ALGORITHM_DEFAULT + } else { + algorithm + } + } + override def supportFileFormatRead( format: ReadFileFormat, fields: Array[StructField], @@ -143,13 +161,7 @@ object CHBackendSettings extends BackendSettingsApi with Logging { } else { outputPartitioning match { case hashPartitioning: HashPartitioning => - hashPartitioning.expressions.foreach( - x => { - if (!x.isInstanceOf[Cast]) { - return false - } - }) - true + hashPartitioning.expressions.exists(x => !x.isInstanceOf[AttributeReference]) case _ => false } @@ -203,4 +215,6 @@ object CHBackendSettings extends BackendSettingsApi with Logging { override def shuffleSupportedCodec(): Set[String] = GLUTEN_CLICKHOUSE_SHUFFLE_SUPPORTED_CODEC override def needOutputSchemaForPlan(): Boolean = true + + override def allowDecimalArithmetic: Boolean = !SQLConf.get.decimalOperationsAllowPrecisionLoss } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala index 7fa21e413435..178f0db116fd 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHIteratorApi.scala @@ -21,12 +21,12 @@ import io.glutenproject.backendsapi.IteratorApi import io.glutenproject.execution._ import io.glutenproject.metrics.{GlutenTimeMetric, IMetrics, NativeMetrics} import io.glutenproject.substrait.plan.PlanNode -import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder} +import io.glutenproject.substrait.rel.{ExtensionTableBuilder, LocalFilesBuilder, SplitInfo} import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat import io.glutenproject.utils.{LogLevelUtil, SubstraitPlanPrinterUtil} import io.glutenproject.vectorized.{CHNativeExpressionEvaluator, CloseableCHColumnBatchIterator, GeneralInIterator, GeneralOutIterator} -import org.apache.spark.{InterruptibleIterator, Partition, SparkConf, SparkContext, TaskContext} +import org.apache.spark.{InterruptibleIterator, SparkConf, SparkContext, TaskContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -41,10 +41,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import java.lang.{Long => JLong} import java.net.URI -import java.util +import java.util.{ArrayList => JArrayList, HashMap => JHashMap, Map => JMap} import scala.collection.JavaConverters._ -import scala.collection.mutable class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { @@ -53,57 +52,47 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { * * @return */ - override def genFilePartition( - index: Int, - partitions: Seq[InputPartition], - partitionSchemas: Seq[StructType], - fileFormats: Seq[ReadFileFormat], - wsCxt: WholeStageTransformContext): BaseGlutenPartition = { - val localFilesNodesWithLocations = partitions.indices.map( - i => - partitions(i) match { - case p: GlutenMergeTreePartition => - ( - ExtensionTableBuilder - .makeExtensionTable(p.minParts, p.maxParts, p.database, p.table, p.tablePath), - SoftAffinityUtil.getNativeMergeTreePartitionLocations(p)) - case f: FilePartition => - val paths = new util.ArrayList[String]() - val starts = new util.ArrayList[JLong]() - val lengths = new util.ArrayList[JLong]() - val partitionColumns = mutable.ArrayBuffer.empty[Map[String, String]] - f.files.foreach { - file => - paths.add(new URI(file.filePath).toASCIIString) - starts.add(JLong.valueOf(file.start)) - lengths.add(JLong.valueOf(file.length)) - // TODO: Support custom partition location - val partitionColumn = mutable.Map.empty[String, String] - partitionColumns.append(partitionColumn.toMap) - } - ( - LocalFilesBuilder.makeLocalFiles( - f.index, - paths, - starts, - lengths, - partitionColumns.map(_.asJava).asJava, - fileFormats(i)), - SoftAffinityUtil.getFilePartitionLocations(f)) - case _ => - throw new UnsupportedOperationException(s"Unsupport operators.") - }) - wsCxt.substraitContext.initLocalFilesNodesIndex(0) - wsCxt.substraitContext.setLocalFilesNodes(localFilesNodesWithLocations.map(_._1)) - val substraitPlan = wsCxt.root.toProtobuf - if (index < 3) { - logOnLevel( - GlutenConfig.getConf.substraitPlanLogLevel, - s"The substrait plan for partition $index:\n${SubstraitPlanPrinterUtil - .substraitPlanToJson(substraitPlan)}" - ) + override def genSplitInfo( + partition: InputPartition, + partitionSchemas: StructType, + fileFormat: ReadFileFormat): SplitInfo = { + partition match { + case p: GlutenMergeTreePartition => + ExtensionTableBuilder + .makeExtensionTable( + p.minParts, + p.maxParts, + p.database, + p.table, + p.tablePath, + SoftAffinityUtil.getNativeMergeTreePartitionLocations(p).toList.asJava) + case f: FilePartition => + val paths = new JArrayList[String]() + val starts = new JArrayList[JLong]() + val lengths = new JArrayList[JLong]() + val partitionColumns = new JArrayList[JMap[String, String]] + f.files.foreach { + file => + paths.add(new URI(file.filePath).toASCIIString) + starts.add(JLong.valueOf(file.start)) + lengths.add(JLong.valueOf(file.length)) + // TODO: Support custom partition location + val partitionColumn = new JHashMap[String, String]() + partitionColumns.add(partitionColumn) + } + val preferredLocations = + SoftAffinityUtil.getFilePartitionLocations(paths.asScala.toArray, f.preferredLocations()) + LocalFilesBuilder.makeLocalFiles( + f.index, + paths, + starts, + lengths, + partitionColumns, + fileFormat, + preferredLocations.toList.asJava) + case _ => + throw new UnsupportedOperationException(s"Unsupported input partition.") } - GlutenPartition(index, substraitPlan, localFilesNodesWithLocations.head._2) } /** @@ -122,7 +111,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { val resIter: GeneralOutIterator = GlutenTimeMetric.millis(pipelineTime) { _ => val transKernel = new CHNativeExpressionEvaluator() - val inBatchIters = new util.ArrayList[GeneralInIterator](inputIterators.map { + val inBatchIters = new JArrayList[GeneralInIterator](inputIterators.map { iter => new ColumnarNativeIterator(genCloseableColumnBatchIterator(iter).asJava) }.asJava) transKernel.createKernelWithBatchIterator(inputPartition.plan, inBatchIters, false) @@ -132,14 +121,17 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics private var outputRowCount = 0L private var outputVectorCount = 0L + private var metricsUpdated = false override def hasNext: Boolean = { val res = resIter.hasNext - if (!res) { + // avoid to collect native metrics more than once, 'hasNext' is a idempotent operation + if (!res && !metricsUpdated) { val nativeMetrics = resIter.getMetrics.asInstanceOf[NativeMetrics] nativeMetrics.setFinalOutputMetrics(outputRowCount, outputVectorCount) updateNativeMetrics(nativeMetrics) updateInputMetrics(inputMetrics) + metricsUpdated = true } res } @@ -177,12 +169,12 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { _ => val transKernel = new CHNativeExpressionEvaluator() val columnarNativeIterator = - new java.util.ArrayList[GeneralInIterator](inputIterators.map { + new JArrayList[GeneralInIterator](inputIterators.map { iter => new ColumnarNativeIterator(genCloseableColumnBatchIterator(iter).asJava) }.asJava) // we need to complete dependency RDD's firstly transKernel.createKernelWithBatchIterator( - rootNode.toProtobuf, + rootNode.toProtobuf.toByteArray, columnarNativeIterator, materializeInput) } @@ -190,13 +182,16 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { val resIter = new Iterator[ColumnarBatch] { private var outputRowCount = 0L private var outputVectorCount = 0L + private var metricsUpdated = false override def hasNext: Boolean = { val res = nativeIterator.hasNext - if (!res) { + // avoid to collect native metrics more than once, 'hasNext' is a idempotent operation + if (!res && !metricsUpdated) { val nativeMetrics = nativeIterator.getMetrics.asInstanceOf[NativeMetrics] nativeMetrics.setFinalOutputMetrics(outputRowCount, outputVectorCount) updateNativeMetrics(nativeMetrics) + metricsUpdated = true } res } @@ -224,11 +219,12 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { /** * Generate closeable ColumnBatch iterator. * + * FIXME: This no longer overrides parent's method + * * @param iter * @return */ - override def genCloseableColumnBatchIterator( - iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { + def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { if (iter.isInstanceOf[CloseableCHColumnBatchIterator]) iter else new CloseableCHColumnBatchIterator(iter) } @@ -237,17 +233,28 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { override def genNativeFileScanRDD( sparkContext: SparkContext, wsCxt: WholeStageTransformContext, - fileFormat: ReadFileFormat, - inputPartitions: Seq[InputPartition], + splitInfos: Seq[SplitInfo], numOutputRows: SQLMetric, numOutputBatches: SQLMetric, scanTime: SQLMetric): RDD[ColumnarBatch] = { val substraitPlanPartition = GlutenTimeMetric.withMillisTime { - // generate each partition of all scan exec - inputPartitions.indices.map( - i => { - genFilePartition(i, Seq(inputPartitions(i)), null, Seq(fileFormat), wsCxt) - }) + splitInfos.zipWithIndex.map { + case (splitInfo, index) => + wsCxt.substraitContext.initSplitInfosIndex(0) + wsCxt.substraitContext.setSplitInfos(Seq(splitInfo)) + val substraitPlan = wsCxt.root.toProtobuf + if (index == 0) { + logOnLevel( + GlutenConfig.getConf.substraitPlanLogLevel, + s"The substrait plan for partition $index:\n${SubstraitPlanPrinterUtil + .substraitPlanToJson(substraitPlan)}" + ) + } + GlutenPartition( + index, + substraitPlan.toByteArray, + splitInfo.preferredLocations().asScala.toArray) + } }(t => logInfo(s"Generating the Substrait plan took: $t ms.")) new NativeFileScanColumnarRDD( @@ -260,8 +267,6 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil { /** Compute for BroadcastBuildSideRDD */ override def genBroadcastBuildSideIterator( - split: Partition, - context: TaskContext, broadcasted: Broadcast[BuildSideRelation], broadCastContext: BroadCastHashJoinContext): Iterator[ColumnarBatch] = { CHBroadcastBuildSideCache.getOrBuildBroadcastHashTable(broadcasted, broadCastContext) diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHMetricsApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHMetricsApi.scala index ad290784218a..dcae0ad9e9ea 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHMetricsApi.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHMetricsApi.scala @@ -26,14 +26,15 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import java.{lang, util} +import java.lang.{Long => JLong} +import java.util.{List => JList, Map => JMap} class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { override def metricsUpdatingFunction( child: SparkPlan, - relMap: util.HashMap[lang.Long, util.ArrayList[lang.Long]], - joinParamsMap: util.HashMap[lang.Long, JoinParams], - aggParamsMap: util.HashMap[lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { MetricsUtil.updateNativeMetrics(child, relMap, joinParamsMap, aggParamsMap) } @@ -154,6 +155,8 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { "extraTime" -> SQLMetrics.createTimingMetric(sparkContext, "extra operators time"), "inputWaitTime" -> SQLMetrics.createTimingMetric(sparkContext, "time of waiting for data"), "outputWaitTime" -> SQLMetrics.createTimingMetric(sparkContext, "time of waiting for output"), + "resizeInputRows" -> SQLMetrics.createMetric(sparkContext, "number of resize input rows"), + "resizeOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of resize output rows"), "preProjectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time of preProjection"), "aggregatingTime" -> @@ -335,5 +338,5 @@ class CHMetricsApi extends MetricsApi with Logging with LogLevelUtil { Map.empty override def genGenerateTransformerMetricsUpdater( - metrics: Map[String, SQLMetric]): MetricsUpdater = new NoopMetricsUpdater + metrics: Map[String, SQLMetric]): MetricsUpdater = NoopMetricsUpdater } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala index 7607440226ef..60c19a469ddf 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHSparkPlanExecApi.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.ColumnarAQEShuffleReadExec +import org.apache.spark.sql.execution.adaptive.AQEShuffleReadExec import org.apache.spark.sql.execution.datasources.GlutenWriterColumnarRules.NativeWritePostRule import org.apache.spark.sql.execution.datasources.v1.ClickHouseFileIndex import org.apache.spark.sql.execution.datasources.v2.BatchScanExec @@ -49,13 +49,15 @@ import org.apache.spark.sql.execution.joins.{BuildSideRelation, ClickHouseBuildS import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.utils.CHExecUtil import org.apache.spark.sql.extension.ClickHouseAnalysis +import org.apache.spark.sql.extension.RewriteDateTimestampComparisonRule import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.common.collect.Lists import org.apache.commons.lang3.ClassUtils -import java.{lang, util} +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.mutable.ArrayBuffer @@ -64,7 +66,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { /** Transform GetArrayItem to Substrait. */ override def genGetArrayItemExpressionNode( substraitExprName: String, - functionMap: java.util.HashMap[String, java.lang.Long], + functionMap: JMap[String, JLong], leftNode: ExpressionNode, rightNode: ExpressionNode, original: GetArrayItem): ExpressionNode = { @@ -286,10 +288,10 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { Seq(ProjectExecTransformer(child.output ++ appendedProjections, wt.child))) case w: WholeStageCodegenExec => w.withNewChildren(Seq(ProjectExec(child.output ++ appendedProjections, w.child))) - case columnarAQEShuffleReadExec: ColumnarAQEShuffleReadExec => + case r: AQEShuffleReadExec if r.supportsColumnar => // when aqe is open // TODO: remove this after pushdowning preprojection - wrapChild(columnarAQEShuffleReadExec) + wrapChild(r) case r2c: RowToCHNativeColumnarExec => wrapChild(r2c) case union: UnionExecTransformer => @@ -328,7 +330,12 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { * @return */ override def genExtendedAnalyzers(): List[SparkSession => Rule[LogicalPlan]] = { - List(spark => new ClickHouseAnalysis(spark, spark.sessionState.conf)) + val analyzers = List(spark => new ClickHouseAnalysis(spark, spark.sessionState.conf)) + if (GlutenConfig.getConf.enableDateTimestampComparison) { + analyzers :+ (spark => new RewriteDateTimestampComparisonRule(spark, spark.sessionState.conf)) + } else { + analyzers + } } /** @@ -371,6 +378,15 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { CHEqualNullSafeTransformer(substraitExprName, left, right, original) } + override def genStringTranslateTransformer( + substraitExprName: String, + srcExpr: ExpressionTransformer, + matchingExpr: ExpressionTransformer, + replaceExpr: ExpressionTransformer, + original: StringTranslate): ExpressionTransformer = { + CHStringTranslateTransformer(substraitExprName, srcExpr, matchingExpr, replaceExpr, original) + } + override def genStringLocateTransformer( substraitExprName: String, first: ExpressionTransformer, @@ -436,9 +452,9 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { /** Generate window function node */ override def genWindowFunctionsNode( windowExpression: Seq[NamedExpression], - windowExpressionNodes: util.ArrayList[WindowFunctionNode], + windowExpressionNodes: JList[WindowFunctionNode], originalInputAttributes: Seq[Attribute], - args: util.HashMap[String, lang.Long]): Unit = { + args: JMap[String, JLong]): Unit = { windowExpression.map { windowExpr => @@ -451,7 +467,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { val frame = aggWindowFunc.frame.asInstanceOf[SpecifiedWindowFrame] val windowFunctionNode = ExpressionBuilder.makeWindowFunction( WindowFunctionsBuilder.create(args, aggWindowFunc).toInt, - new util.ArrayList[ExpressionNode](), + new JArrayList[ExpressionNode](), columnName, ConverterUtils.getTypeNode(aggWindowFunc.dataType, aggWindowFunc.nullable), WindowExecTransformer.getFrameBound(frame.upper), @@ -467,7 +483,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { throw new UnsupportedOperationException(s"Not currently supported: $aggregateFunc.") } - val childrenNodeList = new util.ArrayList[ExpressionNode]() + val childrenNodeList = new JArrayList[ExpressionNode]() aggregateFunc.children.foreach( expr => childrenNodeList.add( @@ -505,7 +521,7 @@ class CHSparkPlanExecApi extends SparkPlanExecApi { } } - val childrenNodeList = new util.ArrayList[ExpressionNode]() + val childrenNodeList = new JArrayList[ExpressionNode]() childrenNodeList.add( ExpressionConverter .replaceWithExpressionTransformer( diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala index 5043b68b9654..5a6e6647fff7 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/backendsapi/clickhouse/CHTransformerApi.scala @@ -19,9 +19,9 @@ package io.glutenproject.backendsapi.clickhouse import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.{BackendsApiManager, TransformerApi} import io.glutenproject.execution.CHHashAggregateExecTransformer -import io.glutenproject.expression.ExpressionConverter +import io.glutenproject.expression.{ConverterUtils, ExpressionConverter} import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.substrait.expression.{CastNode, ExpressionBuilder, ExpressionNode, SelectionNode} +import io.glutenproject.substrait.expression.{BooleanLiteralNode, ExpressionBuilder, ExpressionNode, SelectionNode} import io.glutenproject.utils.{CHInputPartitionsUtil, ExpressionDocUtil} import org.apache.spark.internal.Logging @@ -33,8 +33,11 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} import org.apache.spark.sql.execution.datasources.v1.ClickHouseFileIndex +import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet +import com.google.common.collect.Lists + import java.util class CHTransformerApi extends TransformerApi with Logging { @@ -59,7 +62,7 @@ class CHTransformerApi extends TransformerApi with Logging { val node = ExpressionConverter .replaceWithExpressionTransformer(expr, outputAttributes) .doTransform(substraitContext.registeredFunction) - if (!node.isInstanceOf[SelectionNode] && !node.isInstanceOf[CastNode]) { + if (!node.isInstanceOf[SelectionNode]) { // This is should not happen. logDebug("Expressions are not supported in HashPartitioning.") false @@ -81,18 +84,30 @@ class CHTransformerApi extends TransformerApi with Logging { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] = { if (relation.location.isInstanceOf[ClickHouseFileIndex]) { // Generate NativeMergeTreePartition for MergeTree - relation.location.asInstanceOf[ClickHouseFileIndex].partsPartitions + relation.location + .asInstanceOf[ClickHouseFileIndex] + .partsPartitions( + relation, + selectedPartitions, + output, + bucketedScan, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan + ) } else { // Generate FilePartition for Parquet CHInputPartitionsUtil( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan).genInputPartitionSeq() @@ -156,6 +171,27 @@ class CHTransformerApi extends TransformerApi with Logging { if (!nativeConfMap.containsKey(planOptKey)) { nativeConfMap.put(planOptKey, "false") } + + // Respect spark config spark.sql.orc.compression.codec for CH backend + // TODO: consider compression or orc.compression in table options. + val orcCompressionKey = settingPrefix + "output_format_orc_compression_method" + if (!nativeConfMap.containsKey(orcCompressionKey)) { + if (nativeConfMap.containsKey("spark.sql.orc.compression.codec")) { + val compression = nativeConfMap.get("spark.sql.orc.compression.codec").toLowerCase() + compression match { + case "none" => nativeConfMap.put(orcCompressionKey, "none") + case "uncompressed" => nativeConfMap.put(orcCompressionKey, "none") + case "snappy" => nativeConfMap.put(orcCompressionKey, "snappy") + case "zlib" => nativeConfMap.put(orcCompressionKey, "zlib") + case "zstd" => nativeConfMap.put(orcCompressionKey, "zstd") + case "lz4" => nativeConfMap.put(orcCompressionKey, "lz4") + case _ => + throw new UnsupportedOperationException(s"Not supported ORC compression: $compression") + } + } else { + nativeConfMap.put(orcCompressionKey, "snappy") + } + } } override def getSupportExpressionClassName: util.Set[String] = { @@ -203,4 +239,32 @@ class CHTransformerApi extends TransformerApi with Logging { right: ExpressionNode, escapeChar: ExpressionNode): Iterable[ExpressionNode] = List(left, right) + + override def createCheckOverflowExprNode( + args: java.lang.Object, + substraitExprName: String, + childNode: ExpressionNode, + dataType: DecimalType, + nullable: Boolean, + nullOnOverflow: Boolean): ExpressionNode = { + val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionId = ExpressionBuilder.newScalarFunction( + functionMap, + ConverterUtils.makeFuncName( + substraitExprName, + Seq(dataType, BooleanType), + ConverterUtils.FunctionConfig.OPT)) + + // Just make a fake toType value, because native engine cannot accept datatype itself. + val toTypeNodes = + ExpressionBuilder.makeDecimalLiteral(new Decimal().set(0, dataType.precision, dataType.scale)) + val expressionNodes = + Lists.newArrayList(childNode, new BooleanLiteralNode(nullOnOverflow), toTypeNodes) + val typeNode = ConverterUtils.getTypeNode(dataType, nullable) + ExpressionBuilder.makeScalarFunction(functionId, expressionNodes, typeNode) + } + + override def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String = { + throw new UnsupportedOperationException("CH backend does not support this method") + } } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHFilterExecTransformer.scala b/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHFilterExecTransformer.scala index 2e6aa9b02ca5..72e0bd68e85d 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHFilterExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHFilterExecTransformer.scala @@ -28,8 +28,7 @@ import java.util import scala.collection.JavaConverters._ case class CHFilterExecTransformer(condition: Expression, child: SparkPlan) - extends FilterExecTransformerBase(condition, child) - with TransformSupport { + extends FilterExecTransformerBase(condition, child) { override protected def doValidateInternal(): ValidationResult = { val leftCondition = getLeftCondition diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHHashAggregateExecTransformer.scala b/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHHashAggregateExecTransformer.scala index 2ebb80469e46..ad997843eee6 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHHashAggregateExecTransformer.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/execution/CHHashAggregateExecTransformer.scala @@ -327,8 +327,19 @@ case class CHHashAggregateExecTransformer( ConverterUtils.genColumnNameWithExprId(resultAttr) } else { val aggExpr = aggExpressions(columnIndex - groupingExprs.length) + val aggregateFunc = aggExpr.aggregateFunction var aggFunctionName = - AggregateFunctionsBuilder.getSubstraitFunctionName(aggExpr.aggregateFunction).get + if ( + ExpressionMappings.expressionExtensionTransformer.extensionExpressionsMapping.contains( + aggregateFunc.getClass) + ) { + ExpressionMappings.expressionExtensionTransformer + .buildCustomAggregateFunction(aggregateFunc) + ._1 + .get + } else { + AggregateFunctionsBuilder.getSubstraitFunctionName(aggregateFunc).get + } ConverterUtils.genColumnNameWithExprId(resultAttr) + "#Partial#" + aggFunctionName } } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/expression/CHExpressionTransformer.scala b/backends-clickhouse/src/main/scala/io/glutenproject/expression/CHExpressionTransformer.scala index 8d6a9f987051..ad667179cc00 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/expression/CHExpressionTransformer.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/expression/CHExpressionTransformer.scala @@ -335,6 +335,39 @@ case class CHTruncTimestampTransformer( } } +case class CHStringTranslateTransformer( + substraitExprName: String, + srcExpr: ExpressionTransformer, + matchingExpr: ExpressionTransformer, + replaceExpr: ExpressionTransformer, + original: StringTranslate) + extends ExpressionTransformer { + + override def doTransform(args: java.lang.Object): ExpressionNode = { + // In CH, translateUTF8 requires matchingExpr and replaceExpr argument have the same length + val matchingNode = matchingExpr.doTransform(args) + val replaceNode = replaceExpr.doTransform(args) + if ( + !matchingNode.isInstanceOf[StringLiteralNode] || + !replaceNode.isInstanceOf[StringLiteralNode] + ) { + throw new UnsupportedOperationException(s"$original not supported yet.") + } + + val matchingLiteral = matchingNode.asInstanceOf[StringLiteralNode].getValue + val replaceLiteral = replaceNode.asInstanceOf[StringLiteralNode].getValue + if (matchingLiteral.length() != replaceLiteral.length()) { + throw new UnsupportedOperationException(s"$original not supported yet.") + } + + GenericExpressionTransformer( + substraitExprName, + Seq(srcExpr, matchingExpr, replaceExpr), + original) + .doTransform(args) + } +} + case class CHStringLocateTransformer( substraitExprName: String, substrExpr: ExpressionTransformer, diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala index f3d1bfd53bdd..5cd44a508a6c 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/BatchScanMetricsUpdater.scala @@ -63,6 +63,7 @@ class BatchScanMetricsUpdater(@transient val metrics: Map[String, SQLMetric]) } object BatchScanMetricsUpdater { - val INCLUDING_PROCESSORS = Array("MergeTreeInOrder", "SubstraitFileSource") - val CH_PLAN_NODE_NAME = Array("MergeTreeInOrder", "SubstraitFileSource") + // in mergetree format, the processor name is `MergeTreeSelect(pool: XXX, algorithm: XXX)` + val INCLUDING_PROCESSORS = Array("MergeTreeSelect(pool", "SubstraitFileSource") + val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource") } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala index ad32526272a2..7985efbf0ee6 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/FileSourceScanMetricsUpdater.scala @@ -67,6 +67,7 @@ class FileSourceScanMetricsUpdater(@transient val metrics: Map[String, SQLMetric } object FileSourceScanMetricsUpdater { - val INCLUDING_PROCESSORS = Array("MergeTreeInOrder", "SubstraitFileSource") - val CH_PLAN_NODE_NAME = Array("MergeTreeInOrder", "SubstraitFileSource") + // in mergetree format, the processor name is `MergeTreeSelect(pool: XXX, algorithm: XXX)` + val INCLUDING_PROCESSORS = Array("MergeTreeSelect(pool", "SubstraitFileSource") + val CH_PLAN_NODE_NAME = Array("MergeTreeSelect(pool", "SubstraitFileSource") } diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/HashAggregateMetricsUpdater.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/HashAggregateMetricsUpdater.scala index b3256c1083ad..fff0f6a49e10 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/HashAggregateMetricsUpdater.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/HashAggregateMetricsUpdater.scala @@ -19,6 +19,8 @@ package io.glutenproject.metrics import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.metric.SQLMetric +import scala.collection.JavaConverters._ + class HashAggregateMetricsUpdater(val metrics: Map[String, SQLMetric]) extends MetricsUpdater with Logging { @@ -69,6 +71,16 @@ class HashAggregateMetricsUpdater(val metrics: Map[String, SQLMetric]) HashAggregateMetricsUpdater.CH_PLAN_NODE_NAME ) + val resizeStep = aggMetricsData.steps.asScala + .flatMap(_.processors.asScala) + .find(s => s.getName.equalsIgnoreCase("Resize")) + if (!resizeStep.isEmpty) { + metrics("resizeInputRows") += resizeStep.get.inputRows + metrics("resizeOutputRows") += aggMetricsData.getOutputRows + // The input rows of the Resize is included in the input rows of the Aggregating + metrics("inputRows") += -resizeStep.get.inputRows + } + currentIdx -= 1 // post projection diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala index c47014ce50f4..425fd126bfd3 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala @@ -23,6 +23,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, Collections => JCollections, List => JList, Map => JMap} + import scala.collection.JavaConverters._ object MetricsUtil extends Logging { @@ -38,7 +41,7 @@ object MetricsUtil extends Logging { case t: TransformSupport => MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) case _ => - MetricsUpdaterTree(new NoopMetricsUpdater, Seq()) + MetricsUpdaterTree(NoopMetricsUpdater, Seq()) } } @@ -56,9 +59,9 @@ object MetricsUtil extends Logging { */ def updateNativeMetrics( child: SparkPlan, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { val mut: MetricsUpdaterTree = treeifyMetricsUpdaters(child) @@ -90,10 +93,10 @@ object MetricsUtil extends Logging { */ def updateTransformerMetrics( mutNode: MetricsUpdaterTree, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - operatorIdx: java.lang.Long, - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + operatorIdx: JLong, + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { imetrics => try { val metrics = imetrics.asInstanceOf[NativeMetrics] @@ -104,7 +107,7 @@ object MetricsUtil extends Logging { s"Updating native metrics failed due to the wrong size of metrics data: " + s"$numNativeMetrics") () - } else if (mutNode.updater.isInstanceOf[NoopMetricsUpdater]) { + } else if (mutNode.updater == NoopMetricsUpdater) { () } else { updateTransformerMetricsInternal( @@ -129,22 +132,23 @@ object MetricsUtil extends Logging { */ def updateTransformerMetricsInternal( mutNode: MetricsUpdaterTree, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - operatorIdx: java.lang.Long, + relMap: JMap[JLong, JList[JLong]], + operatorIdx: JLong, metrics: NativeMetrics, metricsIdx: Int, - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): (java.lang.Long, Int) = { - val nodeMetricsList = new java.util.ArrayList[MetricsData]() + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): (JLong, Int) = { + val nodeMetricsList = new JArrayList[MetricsData]() var curMetricsIdx = metricsIdx relMap .get(operatorIdx) .forEach( - _ => { - nodeMetricsList.add(metrics.metricsDataList.get(curMetricsIdx)) + idx => { + nodeMetricsList.add(metrics.metricsDataList.get(idx.toInt)) curMetricsIdx -= 1 }) + JCollections.reverse(nodeMetricsList) val operatorMetrics = new OperatorMetrics( nodeMetricsList, joinParamsMap.getOrDefault(operatorIdx, null), @@ -155,7 +159,7 @@ object MetricsUtil extends Logging { mutNode.children.foreach { child => - if (!child.updater.isInstanceOf[NoopMetricsUpdater]) { + if (child.updater != NoopMetricsUpdater) { val result = updateTransformerMetricsInternal( child, relMap, @@ -192,10 +196,10 @@ object MetricsUtil extends Logging { val processors = MetricsUtil.getAllProcessorList(metricData) processors.foreach( processor => { - if (!includingMetrics.contains(processor.name)) { + if (!includingMetrics.exists(processor.name.startsWith(_))) { extraTime += (processor.time / 1000L).toLong } - if (planNodeNames.contains(processor.name)) { + if (planNodeNames.exists(processor.name.startsWith(_))) { outputRows += processor.outputRows outputBytes += processor.outputBytes inputRows += processor.inputRows diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHExpressionUtil.scala b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHExpressionUtil.scala index bdd884dac6f9..6b560f0bfa8e 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHExpressionUtil.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHExpressionUtil.scala @@ -153,6 +153,8 @@ case class EncodeDecodeValidator() extends FunctionValidator { object CHExpressionUtil { final val CH_AGGREGATE_FUNC_BLACKLIST: Map[String, FunctionValidator] = Map( + MAX_BY -> DefaultValidator(), + MIN_BY -> DefaultValidator() ) final val CH_BLACKLIST_SCALAR_FUNCTION: Map[String, FunctionValidator] = Map( diff --git a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala index 96682fa95721..b4f73d82f560 100644 --- a/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala +++ b/backends-clickhouse/src/main/scala/io/glutenproject/utils/CHInputPartitionsUtil.scala @@ -33,24 +33,12 @@ case class CHInputPartitionsUtil( relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean) extends Logging { - private val bucketedScan: Boolean = { - if ( - relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined - && !disableBucketedScan - ) { - val spec = relation.bucketSpec.get - val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - bucketColumns.size == spec.bucketColumnNames.size - } else { - false - } - } - def genInputPartitionSeq(): Seq[InputPartition] = { if (bucketedScan) { genBucketedInputPartitionSeq() diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala index 0248dec768f4..8af1a46875ec 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/shuffle/CHColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import io.glutenproject.GlutenConfig +import io.glutenproject.backendsapi.clickhouse.CHBackendSettings import io.glutenproject.memory.alloc.CHNativeMemoryAllocators import io.glutenproject.memory.memtarget.{MemoryTarget, Spiller} import io.glutenproject.vectorized._ @@ -100,7 +101,8 @@ class CHColumnarShuffleWriter[K, V]( localDirs, subDirsPerLocalDir, preferSpill, - spillThreshold + spillThreshold, + CHBackendSettings.shuffleHashAlgorithm ) CHNativeMemoryAllocators.createSpillable( "ShuffleWriter", diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala index 49ad39b7fd97..a70ea9bbd421 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/utils/MergeTreePartsPartitionsUtil.scala @@ -21,9 +21,14 @@ import io.glutenproject.execution.GlutenMergeTreePartition import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts import org.apache.spark.sql.execution.datasources.v2.clickhouse.table.ClickHouseTableV2 import org.apache.spark.util.SparkResourceUtil +import org.apache.spark.util.collection.BitSet import scala.collection.mutable.ArrayBuffer @@ -116,4 +121,275 @@ object MergeTreePartsPartitionsUtil extends Logging { closePartition() partitions } + + def getMergeTreePartsPartitions( + relation: HadoopFsRelation, + selectedPartitions: Array[PartitionDirectory], + output: Seq[Attribute], + bucketedScan: Boolean, + sparkSession: SparkSession, + table: ClickHouseTableV2, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + disableBucketedScan: Boolean): Seq[InputPartition] = { + val partsFiles = table.listFiles() + + val partitions = new ArrayBuffer[InputPartition] + val (database, tableName) = if (table.catalogTable.isDefined) { + (table.catalogTable.get.identifier.database.get, table.catalogTable.get.identifier.table) + } else { + // for file_format.`file_path` + ("default", "file_format") + } + val engine = table.snapshot.metadata.configuration.get("engine").get + // TODO: remove `substring` + val tablePath = table.deltaLog.dataPath.toString.substring(6) + + // bucket table + if (table.bucketOption.isDefined) { + if (bucketedScan) { + genBucketedInputPartitionSeq( + engine, + database, + tableName, + tablePath, + table.bucketOption.get, + partsFiles, + partitions, + optionalBucketSet, + optionalNumCoalescedBuckets, + sparkSession + ) + } else { + genInputPartitionSeqWithBucketTable( + engine, + database, + tableName, + tablePath, + table.bucketOption.get, + partsFiles, + partitions, + optionalBucketSet, + sparkSession) + } + } else { + genNonBuckedInputPartitionSeq( + engine, + database, + tableName, + tablePath, + partsFiles, + partitions, + sparkSession + ) + } + partitions + } + + /** Generate bucket partition */ + def genBucketedInputPartitionSeq( + engine: String, + database: String, + tableName: String, + tablePath: String, + bucketSpec: BucketSpec, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + sparkSession: SparkSession): Unit = { + val bucketGroupParts = partsFiles.groupBy(p => Integer.parseInt(p.bucketNum)) + + val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { + val bucketSet = optionalBucketSet.get + bucketGroupParts.filter(f => bucketSet.get(f._1)) + } else { + bucketGroupParts + } + + if (optionalNumCoalescedBuckets.isDefined) { + throw new UnsupportedOperationException( + "Currently CH backend can't support coalesced buckets.") + } + Seq.tabulate(bucketSpec.numBuckets) { + bucketId => + val currBucketParts = prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty) + if (!currBucketParts.isEmpty) { + var currentMinPartsNum = Long.MaxValue + var currentMaxPartsNum = -1L + var currTableName = tableName + "_" + currBucketParts(0).bucketNum + var currTablePath = tablePath + "/" + currBucketParts(0).bucketNum + currBucketParts.foreach( + p => { + if (currentMinPartsNum >= p.minBlockNumber) currentMinPartsNum = p.minBlockNumber + if (currentMaxPartsNum <= p.maxBlockNumber) currentMaxPartsNum = p.maxBlockNumber + }) + if (currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + bucketId, + engine, + database, + currTableName, + currTablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + } + } + } + + /** Generate partition from the bucket table */ + def genInputPartitionSeqWithBucketTable( + engine: String, + database: String, + tableName: String, + tablePath: String, + bucketSpec: BucketSpec, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + optionalBucketSet: Option[BitSet], + sparkSession: SparkSession): Unit = { + val bucketGroupParts = partsFiles.groupBy(p => Integer.parseInt(p.bucketNum)) + + val prunedFilesGroupedToBuckets = if (optionalBucketSet.isDefined) { + val bucketSet = optionalBucketSet.get + bucketGroupParts.filter(f => bucketSet.get(f._1)) + } else { + bucketGroupParts + } + + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + + def closePartition( + currTableName: String, + currTablePath: String, + currentMinPartsNum: Long, + currentMaxPartsNum: Long): Unit = { + if (currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + partitions.size, + engine, + database, + currTableName, + currTablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + } + + Seq.tabulate(bucketSpec.numBuckets) { + bucketId => + val currBucketParts = prunedFilesGroupedToBuckets.getOrElse(bucketId, Seq.empty) + if (!currBucketParts.isEmpty) { + var currentMinPartsNum = Long.MaxValue + var currentMaxPartsNum = -1L + var currentSize = 0L + var currTableName = tableName + "_" + currBucketParts(0).bucketNum + var currTablePath = tablePath + "/" + currBucketParts(0).bucketNum + + currBucketParts.foreach { + parts => + if (currentSize + parts.bytesOnDisk > maxSplitBytes) { + closePartition(currTableName, currTablePath, currentMinPartsNum, currentMaxPartsNum) + currentMinPartsNum = Long.MaxValue + currentMaxPartsNum = -1L + currentSize = 0L + } + // Add the given file to the current partition. + currentSize += parts.bytesOnDisk + openCostInBytes + if (currentMinPartsNum >= parts.minBlockNumber) { + currentMinPartsNum = parts.minBlockNumber + } + if (currentMaxPartsNum <= parts.maxBlockNumber) { + currentMaxPartsNum = parts.maxBlockNumber + } + } + closePartition(currTableName, currTablePath, currentMinPartsNum, currentMaxPartsNum) + } + } + } + + /** Generate partition from the non-bucket table */ + def genNonBuckedInputPartitionSeq( + engine: String, + database: String, + tableName: String, + tablePath: String, + partsFiles: Seq[AddMergeTreeParts], + partitions: ArrayBuffer[InputPartition], + sparkSession: SparkSession): Unit = { + var currentMinPartsNum = -1L + var currentMaxPartsNum = -1L + var currentSize = 0L + var currentFileCnt = 0L + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentMinPartsNum > 0L && currentMaxPartsNum >= currentMinPartsNum) { + val newPartition = GlutenMergeTreePartition( + partitions.size, + engine, + database, + tableName, + tablePath, + currentMinPartsNum, + currentMaxPartsNum + 1) + partitions += newPartition + } + currentMinPartsNum = -1L + currentMaxPartsNum = -1L + currentSize = 0 + currentFileCnt = 0L + } + + val totalCores = SparkResourceUtil.getTotalCores(sparkSession.sessionState.conf) + val fileCntPerPartition = math.ceil((partsFiles.size * 1.0) / totalCores).toInt + val fileCntThreshold = sparkSession.sessionState.conf + .getConfString( + CHBackendSettings.GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD, + CHBackendSettings.GLUTEN_CLICKHOUSE_FILES_PER_PARTITION_THRESHOLD_DEFAULT + ) + .toInt + + if (fileCntThreshold > 0 && fileCntPerPartition > fileCntThreshold) { + // generate `Seq[InputPartition]` by file count + // Assign files to partitions using "Next Fit Decreasing" + partsFiles.foreach { + parts => + if (currentFileCnt >= fileCntPerPartition) { + closePartition() + } + // Add the given file to the current partition. + currentFileCnt += 1 + if (currentMinPartsNum == -1L) { + currentMinPartsNum = parts.minBlockNumber + } + currentMaxPartsNum = parts.maxBlockNumber + } + } else { + // generate `Seq[InputPartition]` by file size + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val maxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + logInfo( + s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + + s"open cost is considered as scanning $openCostInBytes bytes.") + // Assign files to partitions using "Next Fit Decreasing" + partsFiles.foreach { + parts => + if (currentSize + parts.bytesOnDisk > maxSplitBytes) { + closePartition() + } + // Add the given file to the current partition. + currentSize += parts.bytesOnDisk + openCostInBytes + if (currentMinPartsNum == -1L) { + currentMinPartsNum = parts.minBlockNumber + } + currentMaxPartsNum = parts.maxBlockNumber + } + } + closePartition() + } } 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 3e21d71af723..256613ee6b56 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,13 +78,9 @@ class CHRowSplitter extends GlutenRowSplitter { row: FakeRow, partitionColIndice: Array[Int], hasBucket: Boolean): CHBlockStripes = { - val nextBatch = row.batch - - if (nextBatch.numRows > 0) { - val col = nextBatch.column(0).asInstanceOf[CHColumnVector] - new CHBlockStripes( - CHDatasourceJniWrapper - .splitBlockByPartitionAndBucket(col.getBlockAddress, partitionColIndice, hasBucket)) - } else throw new IllegalStateException + val col = row.batch.column(0).asInstanceOf[CHColumnVector] + new CHBlockStripes( + CHDatasourceJniWrapper + .splitBlockByPartitionAndBucket(col.getBlockAddress, partitionColIndice, hasBucket)) } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala index 458a22fe1a2c..047ec5e0490a 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v1/ClickHouseFileIndexBase.scala @@ -17,15 +17,16 @@ package org.apache.spark.sql.execution.datasources.v1 import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow} import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.delta.{DeltaLog, Snapshot} import org.apache.spark.sql.delta.actions.AddFile import org.apache.spark.sql.delta.files.TahoeFileIndex -import org.apache.spark.sql.execution.datasources.PartitionDirectory +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} import org.apache.spark.sql.execution.datasources.utils.MergeTreePartsPartitionsUtil import org.apache.spark.sql.execution.datasources.v2.clickhouse.table.ClickHouseTableV2 import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet import org.apache.hadoop.fs.{FileStatus, Path} @@ -80,8 +81,24 @@ abstract class ClickHouseFileIndexBase( }) } - def partsPartitions: Seq[InputPartition] = - MergeTreePartsPartitionsUtil.getPartsPartitions(spark, table) + def partsPartitions( + relation: HadoopFsRelation, + selectedPartitions: Array[PartitionDirectory], + output: Seq[Attribute], + bucketedScan: Boolean, + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + disableBucketedScan: Boolean): Seq[InputPartition] = + MergeTreePartsPartitionsUtil.getMergeTreePartsPartitions( + relation, + selectedPartitions, + output, + bucketedScan, + spark, + table, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan) override def refresh(): Unit = {} diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ClickHouseAppendDataExec.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ClickHouseAppendDataExec.scala index d75b93e94b32..ede2427b54db 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ClickHouseAppendDataExec.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ClickHouseAppendDataExec.scala @@ -239,13 +239,14 @@ case class ClickHouseAppendDataExec( starts, lengths, partitionColumns.map(_.asJava).asJava, - ReadFileFormat.UnknownFormat) + ReadFileFormat.UnknownFormat, + List.empty.asJava) val insertOutputNode = InsertOutputBuilder.makeInsertOutputNode( SnowflakeIdWorker.getInstance().nextId(), database, tableName, tablePath) - dllCxt.substraitContext.setLocalFilesNodes(Seq(localFilesNode)) + dllCxt.substraitContext.setSplitInfos(Seq(localFilesNode)) dllCxt.substraitContext.setInsertOutputNode(insertOutputNode) val substraitPlan = dllCxt.root.toProtobuf logWarning(dllCxt.root.toProtobuf.toString) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala index bdbb89ea47e1..d47d2bd2159f 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.v2.clickhouse +import org.apache.spark.sql.catalyst.catalog.BucketSpec + import java.util import scala.collection.JavaConverters.mapAsScalaMapConverter @@ -36,8 +38,10 @@ object ClickHouseConfig { val CLICKHOUSE_WAREHOUSE_DIR = "spark.gluten.sql.columnar.backend.ch.warehouse.dir" - /** Validates specified configurations and returns the normalized key -> value map. */ - def validateConfigurations(allProperties: util.Map[String, String]): Map[String, String] = { + /** Create a mergetree configurations and returns the normalized key -> value map. */ + def createMergeTreeConfigurations( + allProperties: util.Map[String, String], + buckets: Option[BucketSpec]): Map[String, String] = { val configurations = scala.collection.mutable.Map[String, String]() allProperties.asScala.foreach(configurations += _) if (!configurations.contains("metadata_path")) { @@ -63,6 +67,13 @@ object ClickHouseConfig { if (!configurations.contains("is_distribute")) { configurations += ("is_distribute" -> "true") } + + if (buckets.isDefined) { + val bucketSpec = buckets.get + configurations += ("numBuckets" -> bucketSpec.numBuckets.toString) + configurations += ("bucketColumnNames" -> bucketSpec.bucketColumnNames.mkString(",")) + configurations += ("sortColumnNames" -> bucketSpec.sortColumnNames.mkString(",")) + } configurations.toMap } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala index de51ccda6fb5..0fc1fcd16cb6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala @@ -92,13 +92,16 @@ class ClickHouseSparkCatalog writeOptions: Map[String, String], sourceQuery: Option[DataFrame], operation: TableCreationModes.CreationMode): Table = { - val tableProperties = ClickHouseConfig.validateConfigurations(allTableProperties) val (partitionColumns, maybeBucketSpec) = SparkShimLoader.getSparkShims.convertPartitionTransforms(partitions) var newSchema = schema var newPartitionColumns = partitionColumns var newBucketSpec = maybeBucketSpec + // Delta does not support bucket feature, so save the bucket infos into properties if exists. + val tableProperties = + ClickHouseConfig.createMergeTreeConfigurations(allTableProperties, newBucketSpec) + val isByPath = isPathIdentifier(ident) val location = if (isByPath) { Option(ident.name()) @@ -146,22 +149,31 @@ class ClickHouseSparkCatalog loadedNewTable match { case v: ClickHouseTableV2 => // TODO: remove this operation after implementing write mergetree into table - ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile(spark.sessionState.newHadoopConf(), v) - v.refresh() + scanMergeTreePartsToAddFile(v) case _ => } loadedNewTable } + def scanMergeTreePartsToAddFile(clickHouseTableV2: ClickHouseTableV2): Unit = { + val (pathFilter, isBucketTable) = if (clickHouseTableV2.bucketOption.isDefined) { + ("/[0-9]*/*_[0-9]*_[0-9]*_[0-9]*", true) + } else { + ("/*_[0-9]*_[0-9]*_[0-9]*", false) + } + ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile( + spark.sessionState.newHadoopConf(), + clickHouseTableV2, + pathFilter, + isBucketTable) + clickHouseTableV2.refresh() + } + /** Performs checks on the parameters provided for table creation for a ClickHouse table. */ private def verifyTableAndSolidify( tableDesc: CatalogTable, query: Option[LogicalPlan]): CatalogTable = { - if (tableDesc.bucketSpec.isDefined) { - throw new UnsupportedOperationException("Do not support Bucketing") - } - val schema = query .map { plan => @@ -215,8 +227,7 @@ class ClickHouseSparkCatalog try { loadTable(ident) match { case v: ClickHouseTableV2 => - ScanMergeTreePartsUtils.scanMergeTreePartsToAddFile(spark.sessionState.newHadoopConf(), v) - v.refresh() + scanMergeTreePartsToAddFile(v) } super.invalidateTable(ident) } catch { 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 f89fb30e9718..1c22aea88faf 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 @@ -36,6 +36,8 @@ case class AddMergeTreeParts( maxBlockNumber: Long, level: Int, dataVersion: Long, + bucketNum: String, + dirName: String, dataChange: Boolean, partition: String = "", defaultCompressionCodec: String = "LZ4", @@ -93,6 +95,8 @@ object AddFileTags { maxBlockNumber: Long, level: Int, dataVersion: Long, + bucketNum: String, + dirName: String, dataChange: Boolean, partition: String = "", defaultCompressionCodec: String = "LZ4", @@ -117,7 +121,9 @@ object AddFileTags { "maxBlockNumber" -> maxBlockNumber.toString, "level" -> level.toString, "dataVersion" -> dataVersion.toString, - "defaultCompressionCodec" -> defaultCompressionCodec + "defaultCompressionCodec" -> defaultCompressionCodec, + "bucketNum" -> bucketNum, + "dirName" -> dirName ) AddFile(name, partitionValues, bytesOnDisk, modificationTime, dataChange, stats, tags) } @@ -142,6 +148,8 @@ object AddFileTags { addFile.tags.get("maxBlockNumber").get.toLong, addFile.tags.get("level").get.toInt, addFile.tags.get("dataVersion").get.toLong, + addFile.tags.get("bucketNum").get, + addFile.tags.get("dirName").get, addFile.dataChange, addFile.tags.get("partition").get, addFile.tags.get("defaultCompressionCodec").get, diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala index d153206e857d..4873796b86c7 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/table/ClickHouseTableV2.scala @@ -162,6 +162,20 @@ case class ClickHouseTableV2( new ClickHouseScanBuilder(spark, this, tableSchema, options) } + lazy val bucketOption: Option[BucketSpec] = { + val tableProperties = properties() + if (tableProperties.containsKey("numBuckets")) { + val numBuckets = tableProperties.get("numBuckets").toInt + val bucketColumnNames: Seq[String] = + tableProperties.get("bucketColumnNames").split(",").toSeq + val sortColumnNames: Seq[String] = + tableProperties.get("sortColumnNames").split(",").toSeq + Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) + } else { + None + } + } + /** Return V1Table. */ override def v1Table: CatalogTable = { if (catalogTable.isEmpty) { @@ -206,7 +220,6 @@ case class ClickHouseTableV2( } val fileIndex = ClickHouseFileIndex(spark, deltaLog, deltaLog.dataPath, this, snapshotToUse, partitionFilters) - var bucketSpec: Option[BucketSpec] = None new HadoopFsRelation( fileIndex, partitionSchema = @@ -217,7 +230,7 @@ case class ClickHouseTableV2( dataSchema = DeltaColumnMapping.dropColumnMappingMetadata( ColumnWithDefaultExprUtils.removeDefaultExpressions( SchemaUtils.dropNullTypeColumns(snapshotToUse.metadata.schema))), - bucketSpec = bucketSpec, + bucketSpec = bucketOption, fileFormat(snapshotToUse.metadata), // `metadata.format.options` is not set today. Even if we support it in future, we shouldn't // store any file system options since they may contain credentials. Hence, it will never @@ -307,7 +320,15 @@ object ClickHouseTableV2 extends Logging { .as[AddFile] .collect() .map(AddFileTags.partsMapToParts) - .sortWith(_.minBlockNumber < _.minBlockNumber) + .sortWith( + (a, b) => { + if (a.bucketNum.nonEmpty) { + (Integer.parseInt(a.bucketNum) < Integer.parseInt(b.bucketNum)) || + (a.minBlockNumber < b.minBlockNumber) + } else { + a.minBlockNumber < b.minBlockNumber + } + }) .toSeq logInfo( s"Get ${allParts.size} parts from path ${tablePath.toString} " + diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala index 48f1011fb6fd..94ecb3525ede 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/utils/ScanMergeTreePartsUtils.scala @@ -31,9 +31,11 @@ object ScanMergeTreePartsUtils extends Logging { def scanMergeTreePartsToAddFile( configuration: Configuration, - clickHouseTableV2: ClickHouseTableV2): Seq[AddFile] = { + clickHouseTableV2: ClickHouseTableV2, + pathFilter: String, + isBucketTable: Boolean): Seq[AddFile] = { // scan parts dir - val scanPath = new Path(clickHouseTableV2.path + "/*_[0-9]*_[0-9]*_[0-9]*") + val scanPath = new Path(clickHouseTableV2.path + pathFilter) val fs = scanPath.getFileSystem(configuration) val fileGlobStatuses = fs.globStatus(scanPath) val allDirSummary = fileGlobStatuses @@ -44,11 +46,26 @@ object ScanMergeTreePartsUtils extends Logging { val sum = fs.getContentSummary(p.getPath) val pathName = p.getPath.getName val pathNameArr = pathName.split("_") - val (partitionId, minBlockNum, maxBlockNum, level) = if (pathNameArr.length == 4) { - (pathNameArr(0), pathNameArr(1).toLong, pathNameArr(2).toLong, pathNameArr(3).toInt) - } else { - ("", 0L, 0L, 0) - } + val (partitionId, bucketNum, minBlockNum, maxBlockNum, level) = + if (pathNameArr.length == 4) { + if (isBucketTable) { + ( + pathNameArr(0), + p.getPath.getParent.getName, + pathNameArr(1).toLong, + pathNameArr(2).toLong, + pathNameArr(3).toInt) + } else { + ( + pathNameArr(0), + "", + pathNameArr(1).toLong, + pathNameArr(2).toLong, + pathNameArr(3).toInt) + } + } else { + ("", "", 0L, 0L, 0) + } ( pathName, partitionId, @@ -56,7 +73,8 @@ object ScanMergeTreePartsUtils extends Logging { maxBlockNum, level, sum.getLength, - p.getModificationTime) + p.getModificationTime, + bucketNum) }) .filter(!_._2.equals("")) @@ -67,13 +85,20 @@ object ScanMergeTreePartsUtils extends Logging { } val finalActions = allDirSummary.map( dir => { + val (filePath, name) = if (isBucketTable) { + ( + clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._8 + "/" + dir._1, + dir._8 + "/" + dir._1) + } else { + (clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._1, dir._1) + } AddFileTags.partsInfoToAddFile( clickHouseTableV2.catalogTable.get.identifier.database.get, clickHouseTableV2.catalogTable.get.identifier.table, clickHouseTableV2.snapshot.metadata.configuration("engine"), - clickHouseTableV2.deltaLog.dataPath.toString + "/" + dir._1, + filePath, "", - dir._1, + name, "", 0L, dir._6, @@ -85,6 +110,8 @@ object ScanMergeTreePartsUtils extends Logging { dir._4, dir._5, dir._3, + dir._8, + dir._1, dataChange = true ) }) 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 0544ad4b90b7..3144522c34e5 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.utils import io.glutenproject.GlutenConfig +import io.glutenproject.backendsapi.clickhouse.CHBackendSettings import io.glutenproject.expression.ConverterUtils import io.glutenproject.row.SparkRowInfo import io.glutenproject.vectorized._ @@ -210,6 +211,7 @@ object CHExecUtil extends Logging { options.setName(nativePartitioning.getShortName) options.setPartitionNum(nativePartitioning.getNumPartitions) options.setExpr(new String(nativePartitioning.getExprList)) + options.setHashAlgorithm(CHBackendSettings.shuffleHashAlgorithm) options.setRequiredFields(if (nativePartitioning.getRequiredFields != null) { new String(nativePartitioning.getRequiredFields) } else { diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out new file mode 100644 index 000000000000..5e74c1cb3cf0 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q01.out @@ -0,0 +1,5 @@ +4 +A|-|F|-|3774200.0|-|5.320753880690033E9|-|5.054096266682795E9|-|5.256751331449255E9|-|25.537587116854997|-|36002.12382901437|-|0.05014459706341336|-|147790 +N|-|F|-|95257.0|-|1.3373779583999987E8|-|1.271323726511998E8|-|1.3228629122944495E8|-|25.30066401062417|-|35521.32691633463|-|0.049394422310757|-|3765 +N|-|O|-|7454519.0|-|1.0505523577969997E10|-|9.979830780624588E9|-|1.0378897125948418E10|-|25.54518413937502|-|36000.3823559628|-|0.05009701285399946|-|291817 +R|-|F|-|3785523.0|-|5.337950526469997E9|-|5.071818532941964E9|-|5.274405503049355E9|-|25.5259438574251|-|35994.0292140309|-|0.049989278561856136|-|148301 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out new file mode 100644 index 000000000000..053518a5a108 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q02.out @@ -0,0 +1,45 @@ +44 +9828.21|-|Supplier#000000647|-|UNITED KINGDOM|-|13120|-|Manufacturer#5|-|x5U7MBZmwfG9|-|33-258-202-4782|-|s the slyly even ideas poach fluffily +9508.37|-|Supplier#000000070|-|FRANCE|-|3563|-|Manufacturer#1|-|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|-|16-821-608-1166|-|ests sleep quickly express ideas. ironic ideas haggle about the final T +9508.37|-|Supplier#000000070|-|FRANCE|-|17268|-|Manufacturer#4|-|INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4|-|16-821-608-1166|-|ests sleep quickly express ideas. ironic ideas haggle about the final T +9453.01|-|Supplier#000000802|-|ROMANIA|-|10021|-|Manufacturer#5|-|,6HYXb4uaHITmtMBj4Ak57Pd|-|29-342-882-6463|-|gular frets. permanently special multipliers believe blithely alongs +9453.01|-|Supplier#000000802|-|ROMANIA|-|13275|-|Manufacturer#4|-|,6HYXb4uaHITmtMBj4Ak57Pd|-|29-342-882-6463|-|gular frets. permanently special multipliers believe blithely alongs +9192.1|-|Supplier#000000115|-|UNITED KINGDOM|-|13325|-|Manufacturer#1|-|nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV|-|33-597-248-1220|-|es across the carefully express accounts boost caref +9032.15|-|Supplier#000000959|-|GERMANY|-|4958|-|Manufacturer#4|-|8grA EHBnwOZhO|-|17-108-642-3106|-|nding dependencies nag furiou +8702.02|-|Supplier#000000333|-|RUSSIA|-|11810|-|Manufacturer#3|-|MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH|-|32-508-202-6136|-|oss the deposits cajole carefully even pinto beans. regular foxes detect alo +8615.5|-|Supplier#000000812|-|FRANCE|-|10551|-|Manufacturer#2|-|8qh4tezyScl5bidLAysvutB,,ZI2dn6xP|-|16-585-724-6633|-|y quickly regular deposits? quickly pending packages after the caref +8615.5|-|Supplier#000000812|-|FRANCE|-|13811|-|Manufacturer#4|-|8qh4tezyScl5bidLAysvutB,,ZI2dn6xP|-|16-585-724-6633|-|y quickly regular deposits? quickly pending packages after the caref +8488.53|-|Supplier#000000367|-|RUSSIA|-|6854|-|Manufacturer#4|-|E Sv9brQVf43Mzz|-|32-458-198-9557|-|ages. carefully final excuses nag finally. carefully ironic deposits abov +8430.52|-|Supplier#000000646|-|FRANCE|-|11384|-|Manufacturer#3|-|IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt|-|16-601-220-5489|-|ites among the always final ideas kindle according to the theodolites. notornis in +8271.39|-|Supplier#000000146|-|RUSSIA|-|4637|-|Manufacturer#5|-|rBDNgCr04x0sfdzD5,gFOutCiG2|-|32-792-619-3155|-|s cajole quickly special requests. quickly enticing theodolites h +8096.98|-|Supplier#000000574|-|RUSSIA|-|323|-|Manufacturer#4|-|2O8 sy9g2mlBOuEjzj0pA2pevk,|-|32-866-246-8752|-|ully after the regular requests. slyly final dependencies wake slyly along the busy deposit +7392.78|-|Supplier#000000170|-|UNITED KINGDOM|-|7655|-|Manufacturer#2|-|RtsXQ,SunkA XHy9|-|33-803-340-5398|-|ake carefully across the quickly +7205.2|-|Supplier#000000477|-|GERMANY|-|10956|-|Manufacturer#5|-|VtaNKN5Mqui5yh7j2ldd5waf|-|17-180-144-7991|-|excuses wake express deposits. furiously careful asymptotes according to the carefull +6820.35|-|Supplier#000000007|-|UNITED KINGDOM|-|13217|-|Manufacturer#5|-|s,4TicNGB4uO6PaSqNBUq|-|33-990-965-2201|-|s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit +6721.7|-|Supplier#000000954|-|FRANCE|-|4191|-|Manufacturer#3|-|P3O5p UFz1QsLmZX|-|16-537-341-8517|-|ect blithely blithely final acco +6329.9|-|Supplier#000000996|-|GERMANY|-|10735|-|Manufacturer#2|-|Wx4dQwOAwWjfSCGupfrM|-|17-447-811-3282|-|ironic forges cajole blithely agai +6173.87|-|Supplier#000000408|-|RUSSIA|-|18139|-|Manufacturer#1|-|qcor1u,vJXAokjnL5,dilyYNmh|-|32-858-724-2950|-|blithely pending packages cajole furiously slyly pending notornis. slyly final +5364.99|-|Supplier#000000785|-|RUSSIA|-|13784|-|Manufacturer#4|-|W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY|-|32-297-653-2203|-|packages boost carefully. express ideas along +5069.27|-|Supplier#000000328|-|GERMANY|-|16327|-|Manufacturer#1|-|SMm24d WG62|-|17-231-513-5721|-|he unusual ideas. slyly final packages a +4941.88|-|Supplier#000000321|-|ROMANIA|-|7320|-|Manufacturer#5|-|pLngFl5yeMcHyov|-|29-573-279-1406|-|y final requests impress s +4672.25|-|Supplier#000000239|-|RUSSIA|-|12238|-|Manufacturer#1|-|XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8|-|32-396-654-6826|-|arls wake furiously deposits. even, regular depen +4586.49|-|Supplier#000000680|-|RUSSIA|-|5679|-|Manufacturer#3|-|UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE|-|32-522-382-1620|-|the regularly regular dependencies. carefully bold excuses under th +4518.31|-|Supplier#000000149|-|FRANCE|-|18344|-|Manufacturer#5|-|pVyWsjOidpHKp4NfKU4yLeym|-|16-660-553-2456|-|ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin +4315.15|-|Supplier#000000509|-|FRANCE|-|18972|-|Manufacturer#2|-|SF7dR8V5pK|-|16-298-154-3365|-|ronic orbits are furiously across the requests. quickly express ideas across the special, bold +3526.53|-|Supplier#000000553|-|FRANCE|-|8036|-|Manufacturer#4|-|a,liVofXbCJ|-|16-599-552-3755|-|lar dinos nag slyly brave +3526.53|-|Supplier#000000553|-|FRANCE|-|17018|-|Manufacturer#3|-|a,liVofXbCJ|-|16-599-552-3755|-|lar dinos nag slyly brave +3294.68|-|Supplier#000000350|-|GERMANY|-|4841|-|Manufacturer#4|-|KIFxV73eovmwhh|-|17-113-181-4017|-|e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick +2972.26|-|Supplier#000000016|-|RUSSIA|-|1015|-|Manufacturer#4|-|YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh|-|32-822-502-4215|-|ously express ideas haggle quickly dugouts? fu +2963.09|-|Supplier#000000840|-|ROMANIA|-|3080|-|Manufacturer#2|-|iYzUIypKhC0Y|-|29-781-337-5584|-|eep blithely regular dependencies. blithely regular platelets sublate alongside o +2221.25|-|Supplier#000000771|-|ROMANIA|-|13981|-|Manufacturer#2|-|lwZ I15rq9kmZXUNhl|-|29-986-304-9006|-|nal foxes eat slyly about the fluffily permanent id +1381.97|-|Supplier#000000104|-|FRANCE|-|18103|-|Manufacturer#3|-|Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m|-|16-434-972-6922|-|gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea +906.07|-|Supplier#000000138|-|ROMANIA|-|8363|-|Manufacturer#4|-|utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw|-|29-533-434-6776|-|ickly unusual requests cajole. accounts above the furiously special excuses +765.69|-|Supplier#000000799|-|RUSSIA|-|11276|-|Manufacturer#2|-|jwFN7ZB3T9sMF|-|32-579-339-1495|-|nusual requests. furiously unusual epitaphs integrate. slyly +727.89|-|Supplier#000000470|-|ROMANIA|-|6213|-|Manufacturer#3|-|XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI|-|29-165-289-1523|-|gular excuses. furiously regular excuses sleep slyly caref +683.07|-|Supplier#000000651|-|RUSSIA|-|4888|-|Manufacturer#4|-|oWekiBV6s,1g|-|32-181-426-4490|-|ly regular requests cajole abou +167.56|-|Supplier#000000290|-|FRANCE|-|2037|-|Manufacturer#1|-|6Bk06GVtwZaKqg01|-|16-675-286-5102|-|the theodolites. ironic, ironic deposits above +91.39|-|Supplier#000000949|-|UNITED KINGDOM|-|9430|-|Manufacturer#2|-|a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV|-|33-332-697-2768|-|pinto beans. carefully express requests hagg +-314.06|-|Supplier#000000510|-|ROMANIA|-|17242|-|Manufacturer#4|-|VmXQl ,vY8JiEseo8Mv4zscvNCfsY|-|29-207-852-3454|-|bold deposits. carefully even d +-820.89|-|Supplier#000000409|-|GERMANY|-|2156|-|Manufacturer#5|-|LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD|-|17-719-517-9836|-|y final, slow theodolites. furiously regular req +-845.44|-|Supplier#000000704|-|ROMANIA|-|9926|-|Manufacturer#5|-|hQvlBqbqqnA5Dgo1BffRBX78tkkRu|-|29-300-896-5991|-|ctions. carefully sly requ +-942.73|-|Supplier#000000563|-|GERMANY|-|5797|-|Manufacturer#1|-|Rc7U1cRUhYs03JD|-|17-108-537-2691|-|slyly furiously final decoys; silent, special realms poach f diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out new file mode 100644 index 000000000000..fb8357ef659a --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q03.out @@ -0,0 +1,11 @@ +10 +223140|-|355369.0698|-|1995-03-14|-|0 +584291|-|354494.7318|-|1995-02-21|-|0 +405063|-|353125.4577|-|1995-03-03|-|0 +573861|-|351238.27699999994|-|1995-03-09|-|0 +554757|-|349181.7426|-|1995-03-14|-|0 +506021|-|321075.58099999995|-|1995-03-10|-|0 +121604|-|318576.41540000006|-|1995-03-07|-|0 +108514|-|314967.0754|-|1995-02-20|-|0 +462502|-|312604.542|-|1995-03-08|-|0 +178727|-|309728.93059999996|-|1995-02-25|-|0 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out new file mode 100644 index 000000000000..766fc7ea4f42 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q04.out @@ -0,0 +1,6 @@ +5 +1-URGENT|-|999 +2-HIGH|-|997 +3-MEDIUM|-|1031 +4-NOT SPECIFIED|-|989 +5-LOW|-|1077 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out new file mode 100644 index 000000000000..b0936e013ebc --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q05.out @@ -0,0 +1,6 @@ +5 +CHINA|-|7822102.999999997 +INDIA|-|6376121.5084999995 +JAPAN|-|6000077.218400001 +INDONESIA|-|5580475.4026999995 +VIETNAM|-|4497840.546600002 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out new file mode 100644 index 000000000000..444c5dc7290b --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q06.out @@ -0,0 +1,2 @@ +1 +1.1803420253400011E7 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out new file mode 100644 index 000000000000..4ef62d4e9848 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q07.out @@ -0,0 +1,5 @@ +4 +FRANCE|-|GERMANY|-|1995|-|4637235.1501 +FRANCE|-|GERMANY|-|1996|-|5224779.573599998 +GERMANY|-|FRANCE|-|1995|-|6232818.703699999 +GERMANY|-|FRANCE|-|1996|-|5557312.1121000005 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out new file mode 100644 index 000000000000..4a6c1930fd55 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q08.out @@ -0,0 +1,3 @@ +2 +1995|-|0.02864874130561755 +1996|-|0.018250279107962144 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out new file mode 100644 index 000000000000..0934cc443ecd --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q09.out @@ -0,0 +1,176 @@ +175 +ALGERIA|-|1998|-|2321785.366697696 +ALGERIA|-|1997|-|3685016.8671991806 +ALGERIA|-|1996|-|4276597.428312926 +ALGERIA|-|1995|-|4418370.424887989 +ALGERIA|-|1994|-|3864849.9593177494 +ALGERIA|-|1993|-|3541051.3923231354 +ALGERIA|-|1992|-|4310013.353297314 +ARGENTINA|-|1998|-|2685983.8006414105 +ARGENTINA|-|1997|-|4242147.80452833 +ARGENTINA|-|1996|-|3907866.997697765 +ARGENTINA|-|1995|-|4605921.504230648 +ARGENTINA|-|1994|-|3542096.1580567244 +ARGENTINA|-|1993|-|3949965.945871686 +ARGENTINA|-|1992|-|4521180.466483507 +BRAZIL|-|1998|-|2778730.394547526 +BRAZIL|-|1997|-|4642037.467789658 +BRAZIL|-|1996|-|4530304.605538634 +BRAZIL|-|1995|-|4502344.872796597 +BRAZIL|-|1994|-|4875806.508928281 +BRAZIL|-|1993|-|4687478.655158408 +BRAZIL|-|1992|-|5035200.053016095 +CANADA|-|1998|-|2194509.047656807 +CANADA|-|1997|-|3482197.946842927 +CANADA|-|1996|-|3712231.268635334 +CANADA|-|1995|-|4014814.8447086867 +CANADA|-|1994|-|4145304.4862727826 +CANADA|-|1993|-|3787069.596131775 +CANADA|-|1992|-|4168009.4262125185 +CHINA|-|1998|-|3398577.9873981243 +CHINA|-|1997|-|6358959.337492856 +CHINA|-|1996|-|6435158.325252927 +CHINA|-|1995|-|6174776.210928064 +CHINA|-|1994|-|6385751.069562026 +CHINA|-|1993|-|5765034.130704248 +CHINA|-|1992|-|6324034.235200413 +EGYPT|-|1998|-|2333148.3330481704 +EGYPT|-|1997|-|3661244.2682996984 +EGYPT|-|1996|-|3765371.225550458 +EGYPT|-|1995|-|4094744.291482351 +EGYPT|-|1994|-|3566508.073745191 +EGYPT|-|1993|-|3725283.76369315 +EGYPT|-|1992|-|3373762.3252903135 +ETHIOPIA|-|1998|-|1953927.2704539902 +ETHIOPIA|-|1997|-|3285786.32113064 +ETHIOPIA|-|1996|-|3525028.7966111703 +ETHIOPIA|-|1995|-|3781674.8841145285 +ETHIOPIA|-|1994|-|3037409.4286103216 +ETHIOPIA|-|1993|-|3008978.2727072844 +ETHIOPIA|-|1992|-|2721203.240953414 +FRANCE|-|1998|-|2604373.8898065495 +FRANCE|-|1997|-|3982872.0631428845 +FRANCE|-|1996|-|3622479.246487785 +FRANCE|-|1995|-|4479939.720681944 +FRANCE|-|1994|-|3531013.206400797 +FRANCE|-|1993|-|4086437.326516616 +FRANCE|-|1992|-|3637792.1427063704 +GERMANY|-|1998|-|3291023.2809489267 +GERMANY|-|1997|-|5139337.326118546 +GERMANY|-|1996|-|4799810.43073009 +GERMANY|-|1995|-|5405785.777712038 +GERMANY|-|1994|-|4555556.43499315 +GERMANY|-|1993|-|4428195.092631204 +GERMANY|-|1992|-|4656148.404258718 +INDIA|-|1998|-|2591288.188878577 +INDIA|-|1997|-|5159562.700538857 +INDIA|-|1996|-|5307258.300371652 +INDIA|-|1995|-|5148208.801210843 +INDIA|-|1994|-|5164001.959532555 +INDIA|-|1993|-|4321398.435788524 +INDIA|-|1992|-|5297703.703198987 +INDONESIA|-|1998|-|3094900.15318282 +INDONESIA|-|1997|-|5719773.037064193 +INDONESIA|-|1996|-|6037238.598229062 +INDONESIA|-|1995|-|5266783.485130789 +INDONESIA|-|1994|-|5470762.866589345 +INDONESIA|-|1993|-|6189826.642484006 +INDONESIA|-|1992|-|4414623.150751896 +IRAN|-|1998|-|3214864.1193034053 +IRAN|-|1997|-|3688049.0721192565 +IRAN|-|1996|-|3621649.2234387854 +IRAN|-|1995|-|4420783.412560043 +IRAN|-|1994|-|4373984.6530368095 +IRAN|-|1993|-|3731301.7799542593 +IRAN|-|1992|-|4417133.370105695 +IRAQ|-|1998|-|2338859.411355956 +IRAQ|-|1997|-|3622681.560582499 +IRAQ|-|1996|-|4762291.873562039 +IRAQ|-|1995|-|4558092.741335291 +IRAQ|-|1994|-|4951604.173189375 +IRAQ|-|1993|-|3830077.986645673 +IRAQ|-|1992|-|3938636.4991397304 +JAPAN|-|1998|-|1849535.076985583 +JAPAN|-|1997|-|4068688.8535843 +JAPAN|-|1996|-|4044774.761344057 +JAPAN|-|1995|-|4793005.800355982 +JAPAN|-|1994|-|4114717.051689451 +JAPAN|-|1993|-|3614468.7382097025 +JAPAN|-|1992|-|4266694.464107132 +JORDAN|-|1998|-|1811488.0784755081 +JORDAN|-|1997|-|2951297.8744189194 +JORDAN|-|1996|-|3302528.3108127 +JORDAN|-|1995|-|3221814.0085915625 +JORDAN|-|1994|-|2417892.0989085087 +JORDAN|-|1993|-|3107641.7733764583 +JORDAN|-|1992|-|3316379.068783395 +KENYA|-|1998|-|2579075.415152621 +KENYA|-|1997|-|2929194.2393937483 +KENYA|-|1996|-|3569129.559210632 +KENYA|-|1995|-|3542889.1139644715 +KENYA|-|1994|-|3983095.3918699673 +KENYA|-|1993|-|3713988.976578933 +KENYA|-|1992|-|3304641.8360172305 +MOROCCO|-|1998|-|1815334.8205032034 +MOROCCO|-|1997|-|3693214.8475239435 +MOROCCO|-|1996|-|4116175.928718384 +MOROCCO|-|1995|-|3515127.1417130576 +MOROCCO|-|1994|-|4003072.102959623 +MOROCCO|-|1993|-|3599199.6739639966 +MOROCCO|-|1992|-|3958335.4259240194 +MOZAMBIQUE|-|1998|-|1620428.7378170486 +MOZAMBIQUE|-|1997|-|2802166.655346226 +MOZAMBIQUE|-|1996|-|2409955.1787294447 +MOZAMBIQUE|-|1995|-|2771602.6354734246 +MOZAMBIQUE|-|1994|-|2548226.2146838196 +MOZAMBIQUE|-|1993|-|2843748.9081963077 +MOZAMBIQUE|-|1992|-|2556501.101767652 +PERU|-|1998|-|2036430.3628828002 +PERU|-|1997|-|4064142.401114619 +PERU|-|1996|-|4068678.575663385 +PERU|-|1995|-|4657694.850584611 +PERU|-|1994|-|4731959.460547456 +PERU|-|1993|-|4144006.6611095574 +PERU|-|1992|-|3754635.006121686 +ROMANIA|-|1998|-|1992773.684887232 +ROMANIA|-|1997|-|2854639.8804400717 +ROMANIA|-|1996|-|3139337.303646003 +ROMANIA|-|1995|-|3222153.3740034285 +ROMANIA|-|1994|-|3222844.3231433732 +ROMANIA|-|1993|-|3488994.033106488 +ROMANIA|-|1992|-|3029274.44639354 +RUSSIA|-|1998|-|2339865.669088989 +RUSSIA|-|1997|-|4153619.5480268304 +RUSSIA|-|1996|-|3772067.3965853318 +RUSSIA|-|1995|-|4704988.859066772 +RUSSIA|-|1994|-|4479082.878186393 +RUSSIA|-|1993|-|4767719.968000547 +RUSSIA|-|1992|-|4533465.554854187 +SAUDI ARABIA|-|1998|-|3386948.9601899004 +SAUDI ARABIA|-|1997|-|5425980.341708716 +SAUDI ARABIA|-|1996|-|5227607.1639614105 +SAUDI ARABIA|-|1995|-|4506731.627395909 +SAUDI ARABIA|-|1994|-|4698658.734425808 +SAUDI ARABIA|-|1993|-|5493626.53336555 +SAUDI ARABIA|-|1992|-|4573560.016888149 +UNITED KINGDOM|-|1998|-|2252021.513909146 +UNITED KINGDOM|-|1997|-|4343926.797442301 +UNITED KINGDOM|-|1996|-|4189476.308943189 +UNITED KINGDOM|-|1995|-|4469569.883577569 +UNITED KINGDOM|-|1994|-|4410094.631914679 +UNITED KINGDOM|-|1993|-|4054677.1017434313 +UNITED KINGDOM|-|1992|-|3978688.875390002 +UNITED STATES|-|1998|-|2238771.5603515483 +UNITED STATES|-|1997|-|4135581.563030852 +UNITED STATES|-|1996|-|3624013.266055848 +UNITED STATES|-|1995|-|3892244.5118940985 +UNITED STATES|-|1994|-|3289224.1137186307 +UNITED STATES|-|1993|-|3626170.192997565 +UNITED STATES|-|1992|-|3993973.494530933 +VIETNAM|-|1998|-|1924313.4846883027 +VIETNAM|-|1997|-|3436195.3714692867 +VIETNAM|-|1996|-|4017288.8905669632 +VIETNAM|-|1995|-|3644054.1472892915 +VIETNAM|-|1994|-|4141277.6721435566 +VIETNAM|-|1993|-|2556114.1622103085 +VIETNAM|-|1992|-|4090524.4854196985 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out new file mode 100644 index 000000000000..661ba845a0e6 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q10.out @@ -0,0 +1,21 @@ +20 +8242|-|Customer#000008242|-|622786.7297|-|6322.09|-|ETHIOPIA|-|P2n4nJhy,UqSo2s43YfSvYJDZ6lk|-|15-792-676-1184|-|slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i +7714|-|Customer#000007714|-|557400.3053|-|9799.98|-|IRAN|-|SnnIGB,SkmnWpX3|-|20-922-418-6024|-|arhorses according to the blithely express re +11032|-|Customer#000011032|-|512500.9640999999|-|8496.93|-|UNITED KINGDOM|-|WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly|-|33-102-772-3533|-|posits-- furiously ironic accounts are again +2455|-|Customer#000002455|-|481592.40530000004|-|2070.99|-|GERMANY|-|RVn1ZSRtLqPlJLIZxvpmsbgC02|-|17-946-225-9977|-|al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro +12106|-|Customer#000012106|-|479414.2133|-|5342.11|-|UNITED STATES|-|wth3twOmu6vy|-|34-905-346-4472|-|ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. +8530|-|Customer#000008530|-|457855.94670000003|-|9734.95|-|MOROCCO|-|GMQyte94oDM7eD7exnkj 4hH9yq3|-|25-736-932-5850|-|slyly asymptotes. quickly final deposits in +13984|-|Customer#000013984|-|446316.5104000001|-|3482.28|-|IRAN|-|qZXwuapCHvxbX|-|20-981-264-2952|-|y unusual courts could wake furiously +1966|-|Customer#000001966|-|444059.03819999995|-|1937.72|-|ALGERIA|-|jPv1 UHra5JLALR5Isci5u0636RoAu7t vH|-|10-973-269-8886|-|the blithely even accounts. final deposits cajole around the blithely final packages. +11026|-|Customer#000011026|-|417913.4142|-|7738.76|-|ALGERIA|-|XorIktoJOAEJkpNNMx|-|10-184-163-4632|-|ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in +8501|-|Customer#000008501|-|412797.51000000007|-|6906.7|-|ARGENTINA|-|776af4rOa mZ66hczs|-|11-317-552-5840|-|y final deposits after the fluffily even accounts are slyly final, regular +1565|-|Customer#000001565|-|412506.00619999995|-|1820.03|-|BRAZIL|-|EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW|-|12-402-178-2007|-|ously regular accounts wake slyly ironic idea +14398|-|Customer#000014398|-|408575.36000000004|-|-602.24|-|UNITED STATES|-|GWRCgIPHajtU21vICVvbJJerFu2cUk|-|34-814-111-5424|-|s. blithely even accounts cajole blithely. even foxes doubt-- +1465|-|Customer#000001465|-|405055.34569999995|-|9365.93|-|INDIA|-|tDRaTC7UgFbBX7VF6cVXYQA0|-|18-807-487-1074|-|s lose blithely ironic, regular packages. regular, final foxes haggle c +12595|-|Customer#000012595|-|401402.2391|-|-6.92|-|INDIA|-|LmeaX5cR,w9NqKugl yRm98|-|18-186-132-3352|-|o the busy accounts. blithely special gifts maintain a +961|-|Customer#000000961|-|401198.17370000004|-|6963.68|-|JAPAN|-|5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn|-|22-989-463-6089|-|e final requests: busily final accounts believe a +14299|-|Customer#000014299|-|400968.3751|-|6595.97|-|RUSSIA|-|7lFczTya0iM1bhEWT|-|32-156-618-1224|-|carefully regular requests. quickly ironic accounts against the ru +623|-|Customer#000000623|-|399883.4257|-|7887.6|-|INDONESIA|-|HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6|-|19-113-202-7085|-|requests. dolphins above the busily regular dependencies cajole after +9151|-|Customer#000009151|-|396562.0295|-|5691.95|-|IRAQ|-|7gIdRdaxB91EVdyx8DyPjShpMD|-|21-834-147-4906|-|ajole fluffily. furiously regular accounts are special, silent account +14819|-|Customer#000014819|-|396271.10360000003|-|7308.39|-|FRANCE|-|w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux|-|16-769-398-7926|-|ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep +13478|-|Customer#000013478|-|395513.13580000005|-|-778.11|-|KENYA|-|9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt|-|24-983-202-8240|-|r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out new file mode 100644 index 000000000000..2a0acdd41cd4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q11.out @@ -0,0 +1,2542 @@ +2541 +12098|-|1.6227681302001953E7 +5134|-|1.5709338600280762E7 +13334|-|1.5023662028076172E7 +17052|-|1.4351644134521484E7 +3452|-|1.4070870250854492E7 +12552|-|1.3332468853271484E7 +1084|-|1.3170427993041992E7 +5797|-|1.3038622729980469E7 +12633|-|1.2892561625549316E7 +403|-|1.2856217222351074E7 +1833|-|1.202458152166748E7 +2084|-|1.1502875112976074E7 +17349|-|1.1354212805175781E7 +18427|-|1.1282385292816162E7 +2860|-|1.1262530243408203E7 +17852|-|1.0934711921264648E7 +9871|-|1.0889253548828125E7 +12231|-|1.0841131487487793E7 +6366|-|1.0759787055236816E7 +12146|-|1.0257362796264648E7 +5043|-|1.0226396083862305E7 +12969|-|1.012577785180664E7 +1504|-|1.000439711505127E7 +14327|-|9981697.163085938 +134|-|9965150.782409668 +6860|-|9805871.418884277 +10624|-|9776138.49609375 +15819|-|9775705.187072754 +3293|-|9674928.096130371 +19865|-|9653766.708618164 +8870|-|9648981.921569824 +15778|-|9636332.747680664 +12360|-|9635023.893310547 +14389|-|9475588.448608398 +3257|-|9451029.524414062 +9476|-|9435207.380859375 +19629|-|9391236.689453125 +7179|-|9386222.462280273 +15723|-|9383900.68359375 +4054|-|9313809.727294922 +2380|-|9307751.31225586 +19084|-|9302916.649169922 +4703|-|9280804.705810547 +18791|-|9267017.754821777 +19994|-|9235972.75805664 +9149|-|9121803.65637207 +15118|-|9120819.403503418 +6116|-|9079369.183227539 +7052|-|9077468.71899414 +14147|-|9069193.912963867 +7305|-|9035228.5390625 +9130|-|9024379.35357666 +16698|-|8991337.720825195 +1553|-|8977225.985046387 +16777|-|8961355.341064453 +1402|-|8953778.921081543 +18963|-|8934063.592407227 +8358|-|8930611.456237793 +17547|-|8860117.219238281 +5128|-|8844222.816467285 +17063|-|8840649.624023438 +15490|-|8833581.16040039 +14761|-|8817240.756347656 +19601|-|8791341.086547852 +16160|-|8740262.78338623 +13597|-|8702669.693637848 +13653|-|8693170.205322266 +16383|-|8691506.236450195 +325|-|8667741.473754883 +8879|-|8667584.401550293 +10564|-|8667098.267211914 +17429|-|8661827.972106934 +17403|-|8643350.085571289 +18294|-|8616583.627380371 +4181|-|8592684.797241211 +13008|-|8567480.8203125 +13211|-|8537000.156066895 +1884|-|8532644.174194336 +11101|-|8530945.055419922 +11562|-|8528028.735031128 +15878|-|8523591.907104492 +834|-|8522135.550048828 +2423|-|8517903.10180664 +15383|-|8513433.094543457 +18119|-|8507612.065429688 +7389|-|8506098.984375 +5016|-|8489783.899841309 +17473|-|8444766.384887695 +6669|-|8428618.723022461 +384|-|8418472.420593262 +12052|-|8411519.051513672 +17562|-|8409022.674499512 +8128|-|8379149.592407227 +13813|-|8374830.797607422 +12800|-|8318626.645629883 +10887|-|8315019.5205078125 +1644|-|8285452.998046875 +16638|-|8274567.789916992 +1394|-|8255140.706726074 +7219|-|8254985.187194824 +13358|-|8253829.867919922 +5562|-|8252365.134216309 +14861|-|8242295.892150879 +15416|-|8196621.351074219 +1963|-|8192206.631652832 +2841|-|8148678.428649902 +6635|-|8122918.347412109 +3609|-|8099812.335205078 +6372|-|8093694.963500977 +5231|-|8091704.0107421875 +8755|-|8085017.173873901 +4071|-|8083755.3486328125 +4977|-|8058501.938415527 +11873|-|8057540.239868164 +12127|-|8051143.096252441 +2401|-|8049525.825378418 +15964|-|8037547.541809082 +10129|-|8030854.880859375 +7605|-|8028622.395629883 +9327|-|8022291.034057617 +11814|-|7983589.842041016 +4646|-|7981660.829956055 +6032|-|7981511.544433594 +1076|-|7977074.816467285 +4043|-|7971930.119384766 +8932|-|7967222.031799316 +13705|-|7953344.6142578125 +16855|-|7923819.0 +3234|-|7920022.0 +17752|-|7901362.887817383 +2097|-|7892993.344177246 +18599|-|7890774.278808594 +19596|-|7874564.710388184 +11392|-|7861172.29296875 +18121|-|7857581.8212890625 +17452|-|7838465.422790527 +6076|-|7821317.0341796875 +15134|-|7804760.25 +8641|-|7802917.367553711 +2134|-|7800185.262374878 +16352|-|7797112.152099609 +19312|-|7775952.218261719 +2895|-|7759327.196136475 +12860|-|7758135.167358398 +153|-|7755681.4609375 +15089|-|7735438.240905762 +14797|-|7725353.121582031 +15946|-|7722773.891845703 +10919|-|7722425.451660156 +9867|-|7721597.614929199 +11881|-|7713136.2685546875 +16552|-|7708517.961303711 +6925|-|7703999.798461914 +12147|-|7703826.796264648 +8923|-|7702690.561279297 +16116|-|7697970.658813477 +10661|-|7673830.010009766 +17094|-|7665368.267822266 +18648|-|7650862.03894043 +12172|-|7641326.5126953125 +15123|-|7633032.4783325195 +4993|-|7600570.908935547 +17162|-|7592062.707092285 +13506|-|7580810.058837891 +3436|-|7575616.171081543 +6271|-|7559794.049926758 +13314|-|7555156.726318359 +17242|-|7550949.617614746 +2753|-|7549573.943603516 +7391|-|7543159.912353516 +7418|-|7541449.544494629 +116|-|7520874.218811035 +12436|-|7520234.376159668 +1181|-|7494798.041259766 +12963|-|7491248.032836914 +213|-|7479470.417358398 +7114|-|7477681.092712402 +18521|-|7476478.258056641 +8973|-|7458603.649963379 +4202|-|7454095.783569336 +12009|-|7442105.203491211 +10609|-|7429346.288696289 +5622|-|7424142.858764648 +5143|-|7422760.191894531 +898|-|7414133.955078125 +12257|-|7408190.745422363 +6740|-|7400350.103759766 +1146|-|7394394.743774414 +5485|-|7378181.939208984 +8437|-|7376353.476928711 +6531|-|7362366.996826172 +16463|-|7362106.593017578 +10412|-|7359552.441833496 +12857|-|7340801.43347168 +12354|-|7332343.0849609375 +7616|-|7320032.390075684 +3426|-|7312340.632568359 +8622|-|7307266.390197754 +6818|-|7304782.037963867 +3705|-|7299379.945068359 +12733|-|7298398.104553223 +1574|-|7293446.1943359375 +10554|-|7289933.552734375 +9363|-|7284647.821655273 +4611|-|7282114.910888672 +7857|-|7266949.0673828125 +9616|-|7265005.353790283 +15860|-|7254466.680541992 +15554|-|7247592.661499023 +3422|-|7247586.549133301 +9134|-|7236404.478027344 +17408|-|7220083.485656738 +15853|-|7219985.026794434 +9266|-|7218517.036376953 +1881|-|7208346.306152344 +10148|-|7205335.997497559 +8860|-|7202401.369445801 +8373|-|7189039.683227539 +10859|-|7188990.444030762 +12670|-|7188177.08215332 +2553|-|7180006.185241699 +19346|-|7176797.25 +1681|-|7160169.945556641 +15225|-|7158861.182128906 +1052|-|7158586.096191406 +77|-|7155531.239624023 +7231|-|7155250.423706055 +12622|-|7137408.324859619 +9814|-|7105363.344665527 +8695|-|7103187.0 +13174|-|7099182.366638184 +6179|-|7095133.949645996 +6451|-|7082495.4892578125 +19860|-|7073206.925537109 +9307|-|7059973.657470703 +7819|-|7055963.275634766 +10556|-|7053491.228027344 +9366|-|7048690.602905273 +12124|-|7040021.477142334 +11476|-|7037906.622802734 +19245|-|7034045.488037109 +7562|-|7030275.7380981445 +12290|-|7020371.895080566 +18118|-|7003396.701599121 +1253|-|7001569.64251709 +8662|-|6999834.2888793945 +1779|-|6997385.874389648 +15386|-|6996871.75869751 +147|-|6989079.989013672 +9562|-|6983076.9216918945 +3958|-|6969833.342041016 +7211|-|6966606.608703613 +12842|-|6923277.385986328 +4368|-|6918784.0388793945 +11131|-|6918656.511962891 +4628|-|6894893.838684082 +5879|-|6881367.098999023 +16586|-|6865931.921081543 +32|-|6852925.70413208 +12119|-|6842773.790405273 +1371|-|6831137.637084961 +6136|-|6827917.031799316 +13857|-|6824240.778198242 +1074|-|6821748.004943848 +1863|-|6821522.124938965 +14597|-|6817385.703491211 +9271|-|6783068.896728516 +6389|-|6781075.729034424 +11703|-|6776538.446533203 +13701|-|6768880.780517578 +14880|-|6763788.366943359 +18428|-|6763670.798339844 +1006|-|6762065.771118164 +4927|-|6756764.979553223 +11659|-|6755246.622680664 +6815|-|6738928.461853027 +14367|-|6733856.956298828 +10703|-|6730936.196655273 +3150|-|6727920.229980469 +9963|-|6725919.502197266 +10438|-|6710153.687133789 +4745|-|6682153.768188477 +15297|-|6681711.262207031 +10848|-|6678666.102096558 +11749|-|6677895.779327393 +16739|-|6675548.987792969 +14915|-|6672248.951416016 +19841|-|6669191.291503906 +628|-|6666893.0322265625 +1650|-|6657178.3419799805 +7069|-|6648672.3673706055 +7108|-|6646445.7658081055 +8887|-|6641656.088623047 +18735|-|6636457.606201172 +3711|-|6632665.1982421875 +2325|-|6630475.801025391 +6484|-|6622964.988525391 +2643|-|6617304.879394531 +7895|-|6615061.113830566 +12840|-|6604848.704223633 +4314|-|6600905.513122559 +19077|-|6591092.154846191 +17476|-|6576028.791809082 +7039|-|6559838.718261719 +8106|-|6558223.037231445 +2391|-|6557120.103363037 +7717|-|6547706.90234375 +12476|-|6546988.352416992 +9155|-|6540793.7138671875 +360|-|6530297.499755859 +6383|-|6529336.0283203125 +4830|-|6518998.776855469 +12600|-|6511549.268859863 +13740|-|6508057.797363281 +4678|-|6507848.167053223 +17815|-|6502284.585021973 +7329|-|6490811.854248047 +13884|-|6490063.071685791 +5147|-|6487069.189453125 +16548|-|6482024.5 +14144|-|6476413.493041992 +10181|-|6474985.073974609 +11031|-|6463307.950134277 +19958|-|6461506.50402832 +6043|-|6458177.505554199 +10060|-|6455476.871154785 +9144|-|6454041.959655762 +7043|-|6448019.869750977 +17346|-|6444307.572265625 +13963|-|6442014.463745117 +7111|-|6441946.990966797 +14140|-|6439955.426147461 +2327|-|6438977.224365234 +14812|-|6437152.762390137 +16755|-|6430895.072143555 +14840|-|6430548.970031738 +14134|-|6422079.210327148 +12655|-|6405496.899597168 +1518|-|6390148.062011719 +9888|-|6385032.905639648 +7387|-|6384005.2341918945 +5393|-|6381083.167236328 +11057|-|6375974.42565918 +16818|-|6368828.832763672 +4576|-|6364925.991455078 +19644|-|6350000.580749512 +139|-|6336065.414428711 +11735|-|6334305.803833008 +10108|-|6332055.9845581055 +15628|-|6329801.422363281 +4349|-|6314949.513183594 +7048|-|6313868.6720581055 +17119|-|6298935.465454102 +18597|-|6291416.160644531 +2488|-|6286371.93951416 +2296|-|6275519.485473633 +4343|-|6272834.280883789 +9958|-|6267714.385986328 +2147|-|6267475.0986328125 +9368|-|6254578.868408203 +13134|-|6252432.635620117 +10119|-|6251456.25 +2093|-|6249342.516357422 +2392|-|6237476.778442383 +17231|-|6233509.230987549 +3773|-|6226654.890014648 +9839|-|6214044.281005859 +19349|-|6213969.458251953 +7869|-|6212751.0 +8158|-|6210279.509033203 +13179|-|6205773.454467773 +2349|-|6205589.162841797 +9052|-|6200729.778686523 +1744|-|6189967.235534668 +597|-|6183103.373657227 +16721|-|6176606.507568359 +7498|-|6176277.166748047 +15157|-|6176222.5 +17524|-|6171107.169433594 +7922|-|6170906.161987305 +6615|-|6170738.276489258 +2706|-|6165991.482055664 +16432|-|6162740.631591797 +16473|-|6162428.176025391 +12825|-|6161595.536499023 +18813|-|6154678.713745117 +1030|-|6146500.111328125 +2571|-|6145772.685852051 +7707|-|6144754.875549316 +10327|-|6137611.892089844 +4710|-|6132346.763671875 +3649|-|6130602.564208984 +4893|-|6128461.310058594 +12844|-|6128191.5185546875 +8794|-|6122690.351074219 +1157|-|6117749.034912109 +10895|-|6112017.556884766 +16166|-|6108251.213562012 +11920|-|6107122.505859375 +5621|-|6102123.578979492 +6141|-|6093826.577270508 +14076|-|6086671.184692383 +15884|-|6080485.499755859 +4814|-|6080337.751464844 +5814|-|6079842.868408203 +1134|-|6078685.3818359375 +19048|-|6071813.099853516 +303|-|6070601.556518555 +15211|-|6065678.1463012695 +1020|-|6054168.1033325195 +11292|-|6052522.887268066 +7134|-|6049865.085952759 +14119|-|6049619.34475708 +2389|-|6042428.9326171875 +5926|-|6034269.631225586 +8553|-|6030922.882080078 +18814|-|6023255.409912109 +12702|-|6023191.234680176 +2644|-|6020931.110595703 +19628|-|6010704.0 +18112|-|6008707.6396484375 +13860|-|6008170.15851593 +1456|-|6005092.087463379 +1633|-|6002068.720092773 +2301|-|6000302.08404541 +10740|-|5999808.927612305 +2630|-|5997008.380023956 +8818|-|5992297.044067383 +10043|-|5990594.109863281 +653|-|5987942.619018555 +6829|-|5985990.5419921875 +15179|-|5977727.351074219 +9663|-|5973524.063964844 +5863|-|5973328.962524414 +3628|-|5966339.861572266 +7618|-|5960156.128417969 +2588|-|5952648.4501953125 +4865|-|5949383.547363281 +4233|-|5944699.3798828125 +13390|-|5944104.823295593 +16321|-|5942714.792358398 +9653|-|5941308.318481445 +18884|-|5925548.072265625 +4394|-|5920927.341247559 +19774|-|5916723.156738281 +1257|-|5914052.5021362305 +2963|-|5911917.549072266 +17157|-|5899572.872558594 +383|-|5884693.19140625 +11709|-|5884134.3427734375 +18135|-|5871431.557067871 +13169|-|5869736.765258789 +2932|-|5868995.733038902 +2888|-|5863229.089355469 +6820|-|5853106.897949219 +18622|-|5850951.483764648 +9066|-|5846052.5634765625 +19840|-|5832251.279296875 +6325|-|5827298.359130859 +14092|-|5823711.827819824 +11532|-|5823277.076904297 +18043|-|5815248.878662109 +3932|-|5809134.717407227 +10364|-|5808371.517333984 +1179|-|5808303.366882324 +11441|-|5799378.315124512 +15767|-|5798263.213623047 +14063|-|5797203.8330078125 +11383|-|5793893.076416016 +10065|-|5781764.231262207 +17785|-|5766838.923736572 +18349|-|5761765.970947266 +14824|-|5760339.707763672 +14699|-|5759490.3134765625 +11628|-|5755137.299621582 +4403|-|5752376.759765625 +13090|-|5751949.335632324 +15332|-|5744606.4169921875 +17573|-|5744383.947875977 +12895|-|5741007.202209473 +13577|-|5739210.0849609375 +16594|-|5732523.799926758 +8497|-|5727509.883544922 +2872|-|5724068.866699219 +16181|-|5721787.9775390625 +6639|-|5712041.176513672 +13095|-|5708798.25 +4535|-|5693341.342163086 +10956|-|5692585.5 +19755|-|5686913.407043457 +12995|-|5682200.377685547 +13157|-|5681454.881713867 +1887|-|5681341.26373291 +18472|-|5680356.0 +19814|-|5679360.15625 +18583|-|5669360.568237305 +3894|-|5664385.625732422 +1740|-|5659552.071777344 +62|-|5659470.315979004 +16532|-|5653779.597351074 +995|-|5648973.3865356445 +7486|-|5646851.936279297 +19007|-|5642710.911071777 +13060|-|5642024.591308594 +12371|-|5635710.356678009 +2280|-|5634077.596435547 +3115|-|5631352.335388184 +11107|-|5631252.5623168945 +5873|-|5629125.75 +14743|-|5628732.584472656 +2224|-|5624746.758911133 +2653|-|5623860.028869629 +17622|-|5623057.4638671875 +14700|-|5615343.8232421875 +14740|-|5613929.609283447 +6403|-|5611924.4900512695 +6896|-|5609169.838256836 +10384|-|5607337.7080078125 +16433|-|5605706.863952637 +5898|-|5604937.6337890625 +4789|-|5600488.560180664 +8132|-|5593107.5 +3838|-|5592313.270019531 +13631|-|5586424.658203125 +11233|-|5585247.120605469 +849|-|5583516.667358398 +14653|-|5581550.6408081055 +14788|-|5580433.0 +18181|-|5578562.896972656 +19815|-|5577102.77746582 +5584|-|5576692.219848633 +5385|-|5576420.109619141 +13780|-|5569028.450683594 +9342|-|5566783.749694824 +19056|-|5566523.983154297 +8189|-|5565694.551269531 +13808|-|5560721.831542969 +10635|-|5560058.317565918 +8304|-|5550784.243652344 +14257|-|5549164.2529296875 +8999|-|5542100.162658691 +19134|-|5539312.38583374 +8360|-|5538031.209716797 +6397|-|5536651.866760254 +2597|-|5525317.7001953125 +8631|-|5515909.300048828 +16729|-|5512663.717041016 +11861|-|5511785.877807617 +16853|-|5511690.148681641 +6341|-|5502790.0634765625 +1312|-|5496649.234619141 +5566|-|5495885.978881836 +12519|-|5490649.905700684 +19032|-|5488104.884399414 +8231|-|5479312.5 +3026|-|5466732.1748046875 +6388|-|5466168.759765625 +15349|-|5464571.4921875 +18985|-|5463897.258300781 +19848|-|5454266.482177734 +17378|-|5453284.826416016 +4000|-|5448690.299926758 +3710|-|5445822.699645996 +13181|-|5439774.144287109 +6420|-|5438325.193359375 +3644|-|5437772.141098022 +1117|-|5437024.868469238 +13027|-|5436968.577270508 +12884|-|5432632.476566315 +12781|-|5429160.972290039 +10084|-|5428231.571533203 +5640|-|5423318.725097656 +16208|-|5422901.193969727 +753|-|5416899.832763672 +4120|-|5413822.516418457 +12348|-|5412061.74206543 +1586|-|5411198.56842041 +2503|-|5411125.597412109 +1381|-|5397789.987182617 +19673|-|5397746.727233887 +19389|-|5394426.345214844 +15532|-|5386361.202636719 +3562|-|5380335.203491211 +19114|-|5375132.916870117 +3204|-|5372215.437072754 +6653|-|5365178.0 +3553|-|5363690.341247559 +12687|-|5361682.239746094 +3279|-|5357505.546936035 +9840|-|5350112.194030762 +8258|-|5347064.642700195 +11387|-|5345284.0 +15210|-|5341117.935058594 +15092|-|5340896.915283203 +6052|-|5339762.264892578 +14043|-|5339587.5546875 +6284|-|5336153.102111816 +6297|-|5332357.802307129 +16254|-|5326607.86340332 +18436|-|5326198.32824707 +14436|-|5325517.143447876 +10368|-|5319982.969528198 +6349|-|5317943.289794922 +19984|-|5317619.8974609375 +19080|-|5310678.432128906 +1403|-|5306771.419189453 +5995|-|5305056.170654297 +13214|-|5299420.238037109 +19293|-|5297054.296508789 +7130|-|5289761.362792969 +9809|-|5286784.634033203 +9273|-|5277924.529724121 +16786|-|5260035.0 +11032|-|5256354.57409668 +17719|-|5246322.274169922 +3400|-|5244203.6376953125 +8278|-|5243718.6767578125 +7873|-|5241168.8916015625 +9357|-|5239695.61920166 +2239|-|5235767.694152832 +18695|-|5223960.379364014 +19271|-|5216498.215576172 +4563|-|5214673.551635742 +1896|-|5210699.938537598 +15755|-|5210481.369506836 +17628|-|5209882.317260742 +5747|-|5206892.341796875 +18057|-|5204907.1826171875 +5075|-|5204413.011657715 +5325|-|5190162.620544434 +17675|-|5183135.307250977 +18818|-|5181295.010986328 +11064|-|5180667.3986206055 +12578|-|5177222.121520996 +3725|-|5169183.75 +11112|-|5161545.787353516 +4749|-|5161499.3388671875 +7632|-|5159180.598449707 +13119|-|5151665.577392578 +5653|-|5140324.559326172 +16134|-|5137482.491760254 +5671|-|5136815.460144043 +18348|-|5132980.987426758 +562|-|5132487.530517578 +2562|-|5126763.857269287 +10304|-|5125180.0 +7622|-|5120719.940734863 +12755|-|5118213.738769531 +1814|-|5100705.401733398 +11269|-|5096671.481262207 +2964|-|5095253.5595703125 +1616|-|5091834.1748046875 +9294|-|5090753.442626953 +16793|-|5085330.739562988 +4999|-|5081651.661071777 +17209|-|5079029.167480469 +7151|-|5078937.689697266 +15522|-|5072469.46105957 +3056|-|5072329.481201172 +7612|-|5068322.744506836 +18453|-|5063893.064819336 +18324|-|5058901.131103516 +12266|-|5058186.75 +19394|-|5056235.616027832 +1713|-|5054967.864990234 +15681|-|5051569.63961792 +8274|-|5043328.125 +18160|-|5043074.653747559 +18253|-|5041571.786499023 +11840|-|5040589.7724609375 +1532|-|5033171.0 +584|-|5031602.589111328 +12382|-|5028900.929260254 +14814|-|5022200.2208862305 +19058|-|5019001.76953125 +4487|-|5016640.704345703 +8482|-|5015444.25 +18476|-|5011136.218017578 +12335|-|5003581.307373047 +4455|-|4997933.337524414 +14355|-|4992823.065795898 +15253|-|4992641.995605469 +14069|-|4983244.096069336 +17843|-|4977294.352844238 +9389|-|4975885.794708252 +14435|-|4971442.158325195 +13254|-|4959481.276245117 +9773|-|4955887.912719727 +7615|-|4952421.346801758 +6476|-|4947250.132507324 +9253|-|4945159.7673950195 +14932|-|4934395.533935547 +13253|-|4932867.584838867 +19322|-|4931525.743225098 +16945|-|4931440.647094727 +731|-|4930191.910827637 +6540|-|4925114.492492676 +5148|-|4923047.790527344 +1934|-|4921197.092590332 +15402|-|4920840.825439453 +17914|-|4919607.1953125 +5416|-|4916041.8515625 +16734|-|4914205.101623535 +14967|-|4900262.231689453 +8706|-|4894595.676269531 +136|-|4891960.934814453 +19494|-|4886028.212036133 +8737|-|4880640.568405151 +7653|-|4879423.8212890625 +4149|-|4875782.34375 +7890|-|4872424.212402344 +11142|-|4871415.393676758 +10386|-|4863623.5 +8603|-|4861814.816345215 +2680|-|4861223.819928408 +4891|-|4858103.90246582 +19236|-|4855097.594238281 +14251|-|4854739.705566406 +18895|-|4853365.020751953 +17134|-|4852040.909912109 +4932|-|4843701.620727539 +10033|-|4841647.833496094 +1383|-|4839144.297821045 +18721|-|4837983.256896973 +8618|-|4833125.3302612305 +17386|-|4831545.862121582 +3790|-|4830282.442749023 +1043|-|4825921.49810791 +12434|-|4822597.65222168 +18385|-|4819643.560058594 +6046|-|4817460.103637695 +5821|-|4814423.387390137 +10836|-|4814303.368774414 +6848|-|4813954.126464844 +6880|-|4804600.252258301 +11249|-|4800116.750732422 +11970|-|4799739.897216797 +14253|-|4796521.322387695 +7782|-|4793227.303466797 +75|-|4790042.787597656 +7076|-|4789347.4599609375 +9566|-|4782531.823425293 +2137|-|4767931.627807617 +2336|-|4763870.767028809 +15362|-|4759043.4904174805 +3284|-|4755048.849243164 +12964|-|4753627.362182617 +1781|-|4752835.270996094 +5454|-|4748342.825683594 +12597|-|4742077.73449707 +19120|-|4733459.977172852 +14884|-|4731499.487548828 +825|-|4730720.185546875 +14683|-|4730482.173583984 +5361|-|4726113.098144531 +12179|-|4725018.062133789 +1461|-|4710954.750732422 +9890|-|4709658.477355957 +13369|-|4705085.355682373 +11612|-|4701627.890991211 +3096|-|4699414.604003906 +10383|-|4697866.486999512 +11293|-|4697335.990661621 +3383|-|4695825.221252441 +6231|-|4694381.618530273 +7396|-|4691319.1111450195 +17827|-|4688797.546325684 +15856|-|4683368.358215332 +8253|-|4678560.997009277 +12327|-|4677984.817504883 +4395|-|4676829.9169921875 +4232|-|4676646.528808594 +14260|-|4670522.9169921875 +15288|-|4669274.073791504 +17526|-|4668545.5458984375 +9884|-|4662693.873291016 +2118|-|4660352.82800293 +4524|-|4653956.5236206055 +19090|-|4650873.041564941 +3928|-|4649359.626403809 +14325|-|4647762.154724121 +15476|-|4643469.148864746 +4179|-|4639931.806640625 +14408|-|4639631.101318359 +19424|-|4634817.4169921875 +3334|-|4633102.5 +9477|-|4628073.531555176 +11803|-|4625396.892623901 +14805|-|4618452.125610352 +463|-|4616307.090332031 +16628|-|4607490.821777344 +3116|-|4604462.978942871 +19962|-|4602949.520690918 +12859|-|4602870.621398926 +12063|-|4600708.30090332 +5648|-|4592273.286437988 +8556|-|4590726.698730469 +15281|-|4589425.466552734 +9414|-|4587426.94152832 +13951|-|4586281.25 +19328|-|4582624.795043945 +15963|-|4579705.430297852 +10773|-|4573276.243286133 +14179|-|4568815.91796875 +1895|-|4563988.246765137 +6408|-|4561496.422485352 +5958|-|4554000.0 +3653|-|4548134.5458984375 +11218|-|4546237.9326171875 +19327|-|4543987.849060059 +9572|-|4535941.016967773 +14556|-|4531464.75 +2475|-|4529761.576538086 +9631|-|4529261.529052734 +1901|-|4528592.450134277 +86|-|4528475.542480469 +9586|-|4527146.243469238 +17361|-|4519098.917480469 +8112|-|4514949.5333862305 +13468|-|4499728.176269531 +18239|-|4497633.723449707 +10215|-|4494553.505859375 +6211|-|4492265.11328125 +836|-|4490945.196228027 +895|-|4489141.567993164 +19542|-|4488393.684387207 +4322|-|4487884.096252441 +2116|-|4486944.625854492 +553|-|4486075.287902832 +2515|-|4485188.394104004 +16286|-|4481470.502471924 +12271|-|4478224.889465332 +16570|-|4465818.176879883 +7995|-|4457574.602233887 +18396|-|4457229.7998046875 +16331|-|4455735.341308594 +18157|-|4452196.66998291 +5271|-|4452040.137268066 +11622|-|4451244.733520508 +4052|-|4446397.302978516 +2864|-|4446008.511779785 +490|-|4442892.223022461 +19837|-|4434172.495391846 +4114|-|4433657.825469971 +11436|-|4433070.259918213 +6085|-|4431306.438903809 +9735|-|4430445.496154785 +17834|-|4416286.344665527 +8157|-|4416116.683654785 +18840|-|4414925.303222656 +13553|-|4412261.64642334 +12562|-|4411183.010620117 +14025|-|4403441.9482421875 +17964|-|4400360.095458984 +636|-|4399863.7763671875 +8390|-|4389024.43762207 +231|-|4387397.207641602 +9699|-|4385890.995589256 +10622|-|4384005.409423828 +14364|-|4383236.970153809 +10580|-|4381533.211486816 +10124|-|4369801.018798828 +10451|-|4368867.477416992 +4673|-|4367113.59765625 +11351|-|4362616.5 +4770|-|4362397.307067871 +12932|-|4362042.668457031 +10603|-|4357216.558532715 +19733|-|4348931.767272949 +4222|-|4348871.946166992 +17319|-|4347687.793029785 +3375|-|4346529.495361328 +14995|-|4338295.762023926 +7675|-|4337499.50592041 +15043|-|4333921.116943359 +4835|-|4332648.0 +4408|-|4332588.947753906 +5559|-|4330577.172637939 +7376|-|4328936.4228515625 +18061|-|4328793.885620117 +2749|-|4328671.324951172 +6628|-|4328501.69720459 +5888|-|4323049.6622924805 +18872|-|4322595.4638671875 +5476|-|4319642.427246094 +1755|-|4318935.650268555 +10623|-|4315822.5732421875 +18775|-|4314677.421386719 +3570|-|4312697.793701172 +11147|-|4310740.499450684 +6071|-|4307612.346496582 +10807|-|4306006.0 +9550|-|4299478.715332031 +657|-|4296794.158325195 +19669|-|4294640.777954102 +8532|-|4290651.657989502 +13469|-|4281715.703979492 +8809|-|4280778.874511719 +11301|-|4276847.8777318 +6147|-|4266879.889434814 +2612|-|4265962.322387695 +15699|-|4256118.787719727 +12300|-|4254409.009460449 +3494|-|4250810.780517578 +11040|-|4250030.305297852 +6190|-|4244046.669616699 +17616|-|4239937.591552734 +7271|-|4234407.0 +14048|-|4226977.31640625 +4456|-|4224684.941772461 +10012|-|4223841.397155762 +11175|-|4223704.178924561 +18675|-|4215406.742797852 +10792|-|4214898.596313477 +10806|-|4209678.513305664 +18749|-|4204787.157592773 +17410|-|4198025.459472656 +8032|-|4195429.946899414 +11094|-|4192304.7974243164 +17582|-|4187341.4150390625 +12246|-|4183231.0021362305 +6640|-|4182968.7302246094 +7346|-|4174707.446411133 +12747|-|4169865.9834594727 +3869|-|4164957.48828125 +13106|-|4161901.9420776367 +10547|-|4159541.3818359375 +15289|-|4156205.864135742 +1679|-|4156156.8388671875 +1126|-|4155593.256286621 +19106|-|4147439.337890625 +9705|-|4144024.211730957 +15324|-|4142518.4815063477 +16544|-|4140375.5946655273 +8812|-|4139322.6481933594 +10772|-|4134101.6876220703 +2800|-|4127149.953125 +15549|-|4124704.596069336 +3607|-|4118697.649291992 +1980|-|4117633.7825927734 +214|-|4113117.2513427734 +19217|-|4104217.701171875 +2460|-|4098577.6850585938 +19156|-|4093864.5733032227 +18359|-|4092727.219848633 +12865|-|4092526.7197875977 +14616|-|4092434.464111328 +908|-|4088856.1779785156 +11791|-|4083804.8350219727 +4157|-|4078345.5336914062 +3857|-|4070873.078857422 +15114|-|4056112.449645996 +395|-|4052997.6689453125 +17456|-|4051457.26171875 +10562|-|4050894.0726013184 +10884|-|4050330.8107910156 +12177|-|4049842.648864746 +15595|-|4040577.5778808594 +15916|-|4036044.5 +7084|-|4035102.84375 +4424|-|4034761.604736328 +10874|-|4031015.730895996 +4740|-|4030403.712890625 +16585|-|4030010.2072143555 +18824|-|4028984.1595458984 +14875|-|4028452.0900878906 +13855|-|4024828.3287963867 +10932|-|4024002.3693847656 +9084|-|4021362.614135742 +14352|-|4018089.609741211 +18086|-|4015180.668823242 +9514|-|4013666.597351074 +15787|-|4013154.5744628906 +714|-|4010249.4217529297 +8811|-|4009588.8416137695 +14386|-|4007210.970336914 +616|-|4004057.293701172 +7460|-|4003412.515625 +866|-|4003182.570739746 +782|-|4001300.0307998657 +8562|-|3999441.6299438477 +1366|-|3994061.0306396484 +2879|-|3993056.5740966797 +16679|-|3992434.9600219727 +17306|-|3990723.359069824 +13140|-|3982817.3999328613 +17942|-|3980857.1481933594 +6572|-|3977676.2143554688 +3578|-|3977523.9614868164 +15802|-|3969947.017211914 +336|-|3967938.2634277344 +9807|-|3964469.5294189453 +12104|-|3964273.4399414062 +4271|-|3962359.2583007812 +6702|-|3961657.408721924 +19763|-|3955582.75 +369|-|3953702.748779297 +4089|-|3953455.662963867 +2593|-|3946153.8842773438 +590|-|3943841.3459472656 +8325|-|3942118.637084961 +158|-|3941881.6287231445 +12054|-|3938362.7642211914 +18330|-|3938303.769104004 +5354|-|3936239.644958496 +8150|-|3925793.3950195312 +8344|-|3921293.4858398438 +6069|-|3921130.6686401367 +4032|-|3920008.510986328 +17939|-|3917750.340209961 +7014|-|3914471.029296875 +2840|-|3913131.778076172 +1868|-|3912987.7229003906 +10975|-|3911920.4897460938 +5374|-|3910802.7037963867 +11128|-|3908156.506958008 +18449|-|3907589.2485351562 +11740|-|3907459.8506774902 +2356|-|3907189.059753418 +5721|-|3901586.026916504 +4231|-|3900779.1184692383 +4352|-|3899933.2465820312 +432|-|3899836.372741699 +15321|-|3899516.5906677246 +10296|-|3897015.3104248047 +5647|-|3895088.204345703 +7386|-|3891916.6166381836 +507|-|3891487.7563476562 +3995|-|3887387.045715332 +4278|-|3882293.9220581055 +18407|-|3880267.991455078 +6127|-|3879166.7860412598 +145|-|3875277.2717285156 +19269|-|3874685.8342285156 +18257|-|3874454.983581543 +9068|-|3869767.7082366943 +576|-|3860007.8736572266 +4860|-|3852861.904525757 +18793|-|3849838.076171875 +15988|-|3847257.015991211 +6891|-|3846386.748565674 +3231|-|3846344.2056274414 +15237|-|3845420.9197998047 +9035|-|3844166.9897460938 +7597|-|3838643.271270752 +16349|-|3837121.6875 +2497|-|3827850.2641296387 +3616|-|3827390.8569335938 +11566|-|3826122.459777832 +18403|-|3822033.0950927734 +2972|-|3821903.605041504 +812|-|3821523.6330566406 +2043|-|3820561.3869628906 +505|-|3818922.0415649414 +8257|-|3815071.929748535 +6084|-|3814195.057220459 +11253|-|3813917.1293945312 +366|-|3812257.971801758 +13632|-|3811601.4802246094 +14298|-|3801412.4647216797 +7092|-|3798729.458129883 +2058|-|3796109.107772827 +14820|-|3791195.745300293 +7157|-|3788690.8326416016 +17211|-|3786030.133605957 +16644|-|3786019.143676758 +15693|-|3783662.0514526367 +2627|-|3782394.4555664062 +11231|-|3782077.5537109375 +12696|-|3781761.7524414062 +8705|-|3778076.8936157227 +16052|-|3771577.163696289 +99|-|3760269.2545776367 +2082|-|3757517.6696777344 +872|-|3750005.232727051 +7126|-|3749138.9309692383 +10302|-|3744475.25 +17122|-|3741012.9901123047 +10080|-|3740106.9815063477 +16021|-|3739611.263671875 +3074|-|3739225.025390625 +3142|-|3738811.042541504 +13213|-|3735116.25 +13442|-|3733132.125854492 +11542|-|3731000.0503845215 +13732|-|3730444.8372802734 +2608|-|3729372.3095703125 +5|-|3725511.3647460938 +19157|-|3723844.697265625 +18231|-|3721707.90524292 +8179|-|3714155.082885742 +12740|-|3708646.8178710938 +11597|-|3706528.71774292 +13968|-|3702376.100830078 +6436|-|3687346.4084472656 +9181|-|3687134.093383789 +564|-|3680200.9166259766 +13464|-|3678406.3126220703 +14084|-|3673790.3356933594 +2755|-|3670593.7201538086 +14284|-|3668640.6528320312 +12178|-|3653392.494445801 +15730|-|3650258.239379883 +5560|-|3649569.574645996 +8594|-|3647140.6193847656 +7032|-|3646439.4858398438 +16846|-|3644843.203063965 +1530|-|3642837.938659668 +3978|-|3639712.0220947266 +2897|-|3639442.5087890625 +16625|-|3636527.7016601562 +12029|-|3636339.9001464844 +16830|-|3633448.560974121 +9597|-|3632662.0404052734 +5533|-|3630338.7088623047 +5181|-|3625965.9406280518 +8131|-|3625738.5552978516 +8560|-|3620761.3450012207 +11860|-|3618746.1837768555 +12008|-|3614604.3872070312 +10737|-|3611990.6669311523 +18208|-|3611596.2127685547 +5119|-|3611038.0553474426 +11958|-|3601654.538269043 +15124|-|3598278.3068847656 +14058|-|3597490.1450195312 +12270|-|3593912.0416259766 +17793|-|3593318.9062805176 +9385|-|3587327.7302246094 +12814|-|3587083.7614746094 +5304|-|3586230.7133483887 +3631|-|3582841.7126464844 +610|-|3581917.384033203 +19317|-|3580412.2911987305 +128|-|3567004.4588623047 +11616|-|3566154.6940307617 +10176|-|3565392.118835449 +7349|-|3564110.6279907227 +1712|-|3560408.479248047 +18860|-|3559340.655517578 +17617|-|3557516.0 +6443|-|3556296.9904174805 +15408|-|3554814.438720703 +16350|-|3554388.6864624023 +17436|-|3554105.249267578 +5740|-|3551324.6645202637 +12181|-|3550218.471008301 +16895|-|3550119.3099975586 +19995|-|3548839.637451172 +4968|-|3548306.836517334 +2257|-|3546692.233581543 +1825|-|3543198.6877441406 +18989|-|3539038.189453125 +18727|-|3536081.463623047 +16165|-|3533789.7639160156 +3249|-|3533709.896118164 +11731|-|3532874.9084472656 +13032|-|3532415.8079223633 +9377|-|3531582.0412597656 +5883|-|3531479.0 +1211|-|3528833.3890686035 +12065|-|3526948.2104492188 +10866|-|3526146.7030029297 +2073|-|3520131.346557617 +2378|-|3512186.1069335938 +16860|-|3509693.15826416 +389|-|3507814.5942382812 +15604|-|3505653.3858947754 +11257|-|3502831.875854492 +1327|-|3502022.69329834 +16602|-|3501074.7788085938 +1493|-|3498809.083404541 +8224|-|3498179.6228027344 +622|-|3497158.3435668945 +3072|-|3495958.7485351562 +1478|-|3494880.3910217285 +3125|-|3494169.7932128906 +2052|-|3488438.0466308594 +8476|-|3487191.348876953 +10735|-|3477740.830078125 +14860|-|3476235.7895507812 +6586|-|3475745.057647705 +5130|-|3472024.5 +7181|-|3471306.362182617 +618|-|3467906.4907226562 +15698|-|3464859.514404297 +17585|-|3462450.507598877 +2548|-|3456856.8356933594 +2632|-|3456230.7512817383 +2882|-|3453986.901489258 +12216|-|3452907.1087646484 +4925|-|3452904.7536621094 +9012|-|3442581.2966308594 +6667|-|3430076.3749694824 +17958|-|3424962.4669189453 +6093|-|3424241.86328125 +10648|-|3417414.0 +1462|-|3413248.7124938965 +2569|-|3412388.917602539 +18616|-|3409880.9376831055 +7368|-|3408036.5661621094 +3110|-|3407374.7145996094 +10824|-|3406819.280822754 +11510|-|3404702.100830078 +4840|-|3397236.5057373047 +4449|-|3396993.7866210938 +1358|-|3396616.348388672 +3885|-|3395817.509765625 +13381|-|3391953.4838867188 +1655|-|3383051.5503845215 +282|-|3381785.3560180664 +4928|-|3374270.3647460938 +3199|-|3372488.6806030273 +16086|-|3370710.6073303223 +8612|-|3362922.5 +19597|-|3360763.9086914062 +8867|-|3354400.0017700195 +4098|-|3353574.403076172 +12617|-|3351499.0365600586 +14365|-|3347296.0 +10443|-|3345493.17388916 +76|-|3342081.729309082 +11585|-|3341941.1455078125 +4383|-|3338960.174133301 +13910|-|3335964.0290527344 +8076|-|3332449.9939575195 +16005|-|3332190.3076171875 +2622|-|3329364.5013427734 +12822|-|3321183.612060547 +17076|-|3320398.0513305664 +5392|-|3320357.1685791016 +18628|-|3319615.9897460938 +13695|-|3318525.9436035156 +10326|-|3318274.172973633 +9109|-|3317833.9587402344 +1489|-|3317620.6997070312 +3378|-|3315948.0 +7738|-|3312979.1337890625 +1844|-|3312277.341064453 +19963|-|3307500.0 +2436|-|3306419.005218506 +886|-|3302180.7923583984 +15475|-|3301693.5424804688 +6327|-|3300680.8302001953 +6050|-|3299460.2420043945 +9876|-|3298409.9957580566 +19586|-|3291131.25 +14349|-|3289862.5565185547 +10993|-|3287980.591003418 +18784|-|3286752.163330078 +1800|-|3285466.403442383 +990|-|3284595.432281494 +3823|-|3281992.824279785 +15737|-|3279305.9416503906 +19518|-|3276759.656616211 +9032|-|3272440.3560791016 +7786|-|3271217.2989501953 +8648|-|3271162.5646362305 +5532|-|3270188.0427856445 +15914|-|3268521.0692749023 +16065|-|3265068.950805664 +11212|-|3264657.0001831055 +13229|-|3262022.2408447266 +15827|-|3260862.626953125 +1582|-|3260340.0 +3827|-|3260093.8667907715 +3546|-|3259243.9357910156 +15849|-|3258918.0603027344 +14856|-|3258379.358276367 +2028|-|3255014.0568847656 +6618|-|3254581.9054260254 +17461|-|3252926.8703918457 +13551|-|3241602.299560547 +19561|-|3239795.201904297 +2276|-|3236172.3681640625 +14203|-|3234649.4604492188 +7757|-|3231351.8129882812 +122|-|3226214.0131225586 +12954|-|3225943.0 +647|-|3224783.795288086 +12383|-|3223989.3447265625 +3831|-|3223126.5568847656 +16836|-|3222260.69430542 +4565|-|3221597.5703125 +19426|-|3218106.4317626953 +17855|-|3217813.072998047 +5624|-|3207777.2919921875 +8368|-|3203376.54964447 +9480|-|3200904.0467834473 +11181|-|3199500.6275177 +8981|-|3197864.0 +16426|-|3195995.9939575195 +1648|-|3195558.815185547 +14404|-|3192729.71875 +17867|-|3188571.1111450195 +18117|-|3183229.0297851562 +14289|-|3182261.478149414 +53|-|3182255.889892578 +15546|-|3180180.031311035 +16245|-|3178277.436401367 +1597|-|3176247.4902648926 +1653|-|3173456.7607421875 +2845|-|3171619.5626220703 +15906|-|3171187.61819458 +18304|-|3168571.456604004 +14068|-|3167367.6510620117 +6837|-|3165012.5779418945 +9446|-|3164446.4759521484 +18889|-|3156140.987121582 +16587|-|3154210.2822875977 +7705|-|3152977.2998657227 +1120|-|3151591.1345214844 +17665|-|3148848.0 +5311|-|3146721.798095703 +14157|-|3144707.3118286133 +7996|-|3131350.9814453125 +8663|-|3130526.3837890625 +18271|-|3127800.8067626953 +6446|-|3125685.914794922 +6972|-|3125007.0802001953 +2572|-|3123186.7198791504 +13536|-|3122527.621398926 +6196|-|3122172.51171875 +9338|-|3121262.5478515625 +11992|-|3118647.639770508 +2580|-|3118284.4561157227 +9098|-|3117494.163696289 +5118|-|3112661.9350585938 +10184|-|3109293.5043945312 +9932|-|3105818.1396484375 +18545|-|3102273.3803100586 +10963|-|3099314.6447753906 +8405|-|3097121.0288085938 +9037|-|3095194.9768066406 +179|-|3091107.3139648438 +1930|-|3090915.8635253906 +17723|-|3090624.7774658203 +4308|-|3089472.851257324 +8702|-|3080129.96484375 +18621|-|3079984.7451171875 +4501|-|3079781.0495910645 +3590|-|3079049.5182495117 +18264|-|3078858.335357666 +15648|-|3078564.2014160156 +5998|-|3073264.0 +16904|-|3072610.725402832 +3794|-|3071333.0126342773 +3147|-|3068485.215576172 +17221|-|3068337.2501220703 +4709|-|3067523.235107422 +18017|-|3066743.440551758 +15613|-|3063987.932067871 +16271|-|3057051.243041992 +13621|-|3054774.5126342773 +12919|-|3054518.4603881836 +12493|-|3050836.1889648438 +15838|-|3050645.8373413086 +3273|-|3048955.118408203 +8324|-|3046011.25 +13628|-|3045324.4122314453 +5522|-|3044408.5 +2202|-|3043131.962585449 +19052|-|3042566.4797973633 +5767|-|3041871.7602539062 +17895|-|3036452.231323242 +12586|-|3036386.393737793 +12425|-|3035041.5849609375 +13517|-|3034351.600341797 +2363|-|3033336.546936035 +15060|-|3032598.603790283 +6764|-|3032591.1453552246 +340|-|3030522.1069335938 +4723|-|3028910.2111816406 +3566|-|3027858.6860351562 +17796|-|3026838.8897094727 +15384|-|3023792.578125 +16336|-|3010813.478668213 +679|-|3010713.344604492 +7554|-|3010667.6899414062 +14553|-|3009756.8935546875 +8379|-|3009745.2450256348 +15436|-|3007499.8440856934 +12471|-|3003991.863304138 +18059|-|3003037.4895629883 +8536|-|3000745.9350585938 +19033|-|2999373.2282714844 +18179|-|2996151.2915039062 +10711|-|2996143.1240844727 +17271|-|2994264.7814331055 +13932|-|2989023.5170898438 +3101|-|2987788.302307129 +14550|-|2977853.6235046387 +3080|-|2977232.5885620117 +14533|-|2976490.5729370117 +14439|-|2975313.2834472656 +9237|-|2973124.6833496094 +1205|-|2971470.2993164062 +12361|-|2963419.493774414 +429|-|2962631.9146728516 +3970|-|2960418.432006836 +8403|-|2957698.399383545 +1098|-|2957514.0899658203 +7932|-|2955046.082672119 +16266|-|2952298.344482422 +19386|-|2948854.533721924 +13147|-|2947037.881072998 +2720|-|2947011.0479125977 +3840|-|2944219.253540039 +13482|-|2942474.947265625 +9436|-|2940396.258544922 +19779|-|2937105.8419189453 +18032|-|2933224.391052246 +7743|-|2932733.780944824 +14620|-|2930766.9780273438 +4606|-|2927832.6963806152 +18076|-|2924134.807067871 +19276|-|2918176.1877441406 +7483|-|2915919.015197754 +8575|-|2915132.6059570312 +11052|-|2913140.9509277344 +17251|-|2908345.850830078 +8788|-|2907935.816467285 +10960|-|2906511.1927490234 +18782|-|2903643.813232422 +19988|-|2897461.6274414062 +726|-|2896009.2184143066 +19942|-|2894251.367553711 +10864|-|2892252.5493164062 +17840|-|2891563.2795410156 +18717|-|2888939.8825683594 +12391|-|2886051.3537597656 +18219|-|2885920.995788574 +15100|-|2883342.3768310547 +2491|-|2880385.682739258 +12389|-|2879696.8638305664 +3880|-|2877770.2478637695 +18579|-|2874542.435546875 +13647|-|2873838.3503112793 +15758|-|2873804.818359375 +12917|-|2873659.5251464844 +18866|-|2873616.267883301 +13894|-|2872986.000732422 +15200|-|2872572.021789551 +9628|-|2872404.590698242 +8568|-|2871598.1217041016 +8389|-|2870238.0225219727 +5788|-|2867210.204788208 +19450|-|2863310.568725586 +9440|-|2863162.8435058594 +16795|-|2860135.2837524414 +19643|-|2858987.7473449707 +1974|-|2856825.715484619 +14622|-|2852089.0698242188 +6885|-|2851437.7064208984 +12532|-|2848992.5150146484 +1087|-|2847858.878173828 +5777|-|2846407.4865112305 +5629|-|2846076.0428466797 +6316|-|2840544.710998535 +12767|-|2840514.233642578 +12134|-|2840036.862548828 +14476|-|2839852.970489502 +803|-|2838388.204772949 +18964|-|2836942.512084961 +6020|-|2833459.096069336 +10401|-|2832688.8435058594 +1323|-|2829964.402770996 +1151|-|2829662.5451660156 +1458|-|2824034.4858398438 +2271|-|2820756.481201172 +18740|-|2814140.8579101562 +7348|-|2811731.043823242 +4281|-|2807190.532775879 +8043|-|2804706.1772460938 +3843|-|2804217.932373047 +7813|-|2802350.8267822266 +347|-|2802245.3862304688 +745|-|2801725.149230957 +10388|-|2799170.4808044434 +18100|-|2793358.5 +19043|-|2789013.861694336 +10644|-|2787796.8673706055 +16170|-|2787402.87109375 +398|-|2782729.127807617 +9370|-|2780078.1205444336 +14504|-|2780035.946166992 +1210|-|2778485.6623535156 +13385|-|2777445.5392456055 +3799|-|2775223.6352539062 +11325|-|2769765.9909057617 +3489|-|2769554.4653320312 +17181|-|2769028.3795166016 +6964|-|2766653.6810913086 +7381|-|2764898.7114257812 +6253|-|2764394.5572509766 +5975|-|2760819.7994384766 +11996|-|2760687.8385620117 +7570|-|2758977.1689453125 +4387|-|2757671.8505859375 +9014|-|2755367.42779541 +9403|-|2748021.6427001953 +11653|-|2739730.983444214 +17697|-|2739312.204559326 +958|-|2738032.0 +18816|-|2737139.9536132812 +14104|-|2735008.6606445312 +15966|-|2732250.2307128906 +17912|-|2724160.959777832 +7089|-|2720170.031982422 +16032|-|2718976.0388183594 +16891|-|2717293.2658081055 +19579|-|2716909.836791992 +17470|-|2715048.8865356445 +12408|-|2712556.5137023926 +4763|-|2711800.934448242 +1138|-|2709709.7604370117 +7363|-|2708414.33203125 +7877|-|2705439.5303649902 +17532|-|2703698.752319336 +10512|-|2701235.9489746094 +11957|-|2700133.234741211 +2455|-|2699593.858276367 +15119|-|2696860.833984375 +9868|-|2696801.4624023438 +14172|-|2695307.4061584473 +16120|-|2689337.8975524902 +13958|-|2679025.2431640625 +15169|-|2676686.086669922 +2648|-|2672232.0336914062 +6164|-|2671317.244873047 +12701|-|2669216.4587402344 +16382|-|2669034.5114746094 +15588|-|2667212.0443725586 +14830|-|2666758.130645752 +9119|-|2665812.2794189453 +1622|-|2665206.492614746 +878|-|2664045.7276306152 +13269|-|2662784.1020507812 +619|-|2655417.6220092773 +18386|-|2653795.01361084 +2501|-|2652260.4766845703 +2310|-|2651631.1859664917 +19420|-|2649395.637451172 +4895|-|2645152.2278060913 +7553|-|2643681.9628601074 +17814|-|2642781.32421875 +16097|-|2642500.0 +10995|-|2640811.183898926 +14895|-|2637733.7329101562 +18546|-|2637026.842102051 +9875|-|2631358.7497558594 +9591|-|2626899.469116211 +6001|-|2625893.671875 +7739|-|2624573.204223633 +10431|-|2624379.52935791 +4544|-|2615313.6489868164 +16387|-|2603195.666015625 +18375|-|2601407.794555664 +8395|-|2598728.4901123047 +18853|-|2593356.426513672 +4900|-|2592813.1634521484 +1302|-|2592197.8223876953 +17032|-|2589806.318359375 +14292|-|2589749.644165039 +43|-|2587359.6740722656 +5221|-|2587024.048187256 +397|-|2579751.5158691406 +17890|-|2579674.2705078125 +12157|-|2575510.537689209 +7340|-|2574645.822418213 +19368|-|2572619.0034484863 +8848|-|2570819.2907714844 +13789|-|2570243.276916504 +14596|-|2568234.1953125 +8408|-|2567434.4887390137 +19726|-|2565750.3982543945 +13964|-|2565579.001647949 +7740|-|2563027.5268554688 +14768|-|2560392.5514526367 +11734|-|2559062.138305664 +10294|-|2558258.104370117 +15032|-|2557926.2736968994 +9127|-|2556379.7252197266 +2181|-|2553175.0 +16653|-|2552229.62109375 +3866|-|2549994.8923339844 +16814|-|2548710.6931152344 +1866|-|2545838.3203125 +3512|-|2532626.9079589844 +4145|-|2529786.1860046387 +12120|-|2528298.5899658203 +644|-|2528123.0612182617 +15379|-|2525181.0533447266 +6392|-|2524063.2072143555 +2652|-|2521456.7431640625 +3363|-|2519202.307800293 +19167|-|2517993.239501953 +16042|-|2516600.0161743164 +2892|-|2511854.4744873047 +5711|-|2509401.8071289062 +14591|-|2506344.669647217 +6564|-|2506277.262084961 +1231|-|2505421.140930176 +5049|-|2502603.0395507812 +14576|-|2501606.7992248535 +10211|-|2500852.2442626953 +293|-|2493168.4423828125 +7371|-|2491134.5950317383 +18154|-|2491047.1584472656 +9494|-|2489825.4729003906 +14836|-|2480432.452850342 +19471|-|2480403.7979125977 +802|-|2478998.2611083984 +12541|-|2477242.539916992 +15065|-|2473563.866333008 +15995|-|2472803.2690429688 +9408|-|2471953.5029296875 +9776|-|2470448.005554199 +17325|-|2468989.0130615234 +3391|-|2468317.634460449 +16123|-|2467022.3071289062 +18758|-|2463797.954498291 +407|-|2460304.3577575684 +6840|-|2456170.6811523438 +9995|-|2455155.4196777344 +3877|-|2453696.583251953 +5817|-|2452493.1763916016 +14122|-|2452226.2705078125 +16699|-|2450274.0151367188 +8921|-|2450116.490966797 +15103|-|2449861.266723633 +7637|-|2449628.7778320312 +3076|-|2443927.3540649414 +6648|-|2443248.844116211 +17116|-|2442263.6349487305 +1645|-|2440838.4868164062 +3181|-|2440017.5247192383 +5966|-|2431558.1162109375 +15882|-|2428947.3501586914 +7529|-|2428381.2517089844 +12836|-|2427897.337097168 +18052|-|2427637.70703125 +13616|-|2426638.5749816895 +16615|-|2424775.1391601562 +18147|-|2424412.7150878906 +4586|-|2424123.8583374023 +14403|-|2423141.969116211 +11606|-|2422794.3608398438 +13526|-|2422212.7685546875 +3677|-|2421404.386291504 +5553|-|2418506.1848449707 +12109|-|2416514.1503601074 +13118|-|2415931.7456054688 +1563|-|2408855.393676758 +16591|-|2408045.473510742 +6411|-|2404918.6111450195 +10272|-|2402834.504119873 +10597|-|2400247.6235961914 +13700|-|2398035.8814697266 +9548|-|2397147.929626465 +14963|-|2395781.1557617188 +13325|-|2390637.514770508 +13864|-|2388067.935913086 +7450|-|2383447.680786133 +9275|-|2382868.42376709 +5829|-|2378038.0029296875 +13437|-|2377806.434326172 +13594|-|2375046.375732422 +11442|-|2374591.08782959 +15619|-|2374052.2861938477 +9063|-|2374035.85546875 +5990|-|2368686.5 +7811|-|2363829.23248291 +9525|-|2362974.501647949 +5597|-|2361031.846282959 +8963|-|2360774.0 +1709|-|2359839.3669433594 +15814|-|2358656.58984375 +17613|-|2357519.0229492188 +5022|-|2354550.4083251953 +17740|-|2354242.7854003906 +3388|-|2351042.2163085938 +13773|-|2348739.107788086 +14467|-|2348665.0895996094 +11544|-|2345324.373779297 +349|-|2344664.1772460938 +10356|-|2340862.6950683594 +18272|-|2338754.6755371094 +4627|-|2337430.850128174 +327|-|2335298.536376953 +19846|-|2332224.6345825195 +10814|-|2330319.610595703 +13102|-|2326122.75 +18867|-|2323972.035675049 +2824|-|2323315.0638427734 +19117|-|2319911.143798828 +1906|-|2319757.5622558594 +245|-|2319450.934753418 +17318|-|2317860.3787231445 +3862|-|2316453.7719726562 +8100|-|2313874.0541992188 +2958|-|2312239.4603881836 +10263|-|2308514.1148986816 +13814|-|2304940.337585449 +9394|-|2303161.745941162 +18080|-|2299416.790157318 +1271|-|2289526.907775879 +3327|-|2278474.467529297 +8740|-|2278405.9985961914 +8119|-|2276428.2103271484 +3368|-|2274373.545776367 +7963|-|2272300.7830810547 +2151|-|2270932.7026367188 +16995|-|2270264.6533203125 +9918|-|2269733.0579223633 +503|-|2268535.1638793945 +16692|-|2256484.5108032227 +793|-|2254198.8298339844 +16455|-|2252361.8503112793 +6644|-|2249521.8873901367 +17280|-|2249437.5 +6813|-|2248982.0513916016 +4674|-|2246915.378051758 +16325|-|2244369.7688598633 +182|-|2243290.0 +4626|-|2242474.3936157227 +10860|-|2241291.5270996094 +14034|-|2241220.8599853516 +2476|-|2240855.1263427734 +4253|-|2239985.7114257812 +3211|-|2239871.0924072266 +1290|-|2233312.97454834 +8479|-|2232189.021713257 +11895|-|2231607.0 +3487|-|2230171.5392456055 +14870|-|2229915.3003845215 +16328|-|2229484.001220703 +18585|-|2228215.4418945312 +7638|-|2228207.9917907715 +5436|-|2225672.3259887695 +14594|-|2223005.0853881836 +4532|-|2215710.971923828 +7586|-|2210562.5312805176 +11870|-|2205182.7026367188 +18487|-|2203653.653564453 +9179|-|2202720.5006713867 +16500|-|2201185.3262939453 +3679|-|2200592.5982666016 +12803|-|2198295.0134277344 +18056|-|2196741.8298339844 +11396|-|2195645.5908203125 +5087|-|2194120.762817383 +8067|-|2192048.619140625 +15357|-|2191646.555786133 +4491|-|2189713.4143066406 +208|-|2189046.713745117 +10958|-|2188766.8798675537 +9126|-|2188410.5 +15084|-|2184327.005859375 +18850|-|2183309.5993652344 +3398|-|2180250.0 +16137|-|2177318.7674560547 +211|-|2174808.926513672 +18422|-|2174381.057739258 +15840|-|2173510.3826904297 +19553|-|2173079.730697632 +8221|-|2169992.1791992188 +17000|-|2169611.2247314453 +6755|-|2168505.068206787 +10817|-|2167710.663925171 +8327|-|2167650.643310547 +543|-|2167367.964477539 +4553|-|2163371.47265625 +15019|-|2162287.9272460938 +334|-|2162178.43460083 +8516|-|2161478.9765625 +11349|-|2158941.968811035 +3902|-|2157027.8850097656 +14731|-|2155302.2080078125 +326|-|2153380.1381835938 +11403|-|2151242.3597717285 +11657|-|2150446.0576171875 +9496|-|2149218.9655151367 +8110|-|2149120.1103515625 +5153|-|2148527.3208618164 +884|-|2148324.8961868286 +8637|-|2146185.1928710938 +2364|-|2145790.7607421875 +12386|-|2145001.3764572144 +10133|-|2144903.972137451 +9895|-|2143324.86328125 +13755|-|2142539.4653320312 +4327|-|2138501.4697265625 +3369|-|2137408.7163085938 +5815|-|2136985.0 +19357|-|2132657.337890625 +2675|-|2124158.6796875 +17869|-|2123991.639678955 +11702|-|2122132.93850708 +17257|-|2117850.6005859375 +9952|-|2116686.3525390625 +3881|-|2111457.2221069336 +10951|-|2111185.603942871 +2128|-|2109702.3321533203 +6699|-|2106578.418273926 +3155|-|2103636.646118164 +16649|-|2101956.143798828 +15257|-|2100297.686126709 +9978|-|2099566.5798339844 +16810|-|2098301.3759765625 +10653|-|2093388.7595214844 +10476|-|2092766.402709961 +10883|-|2087495.2159423828 +9704|-|2086967.6427612305 +1119|-|2085182.7796020508 +19139|-|2079788.3895263672 +2144|-|2078391.1307373047 +9135|-|2076377.7429199219 +18548|-|2075584.3802490234 +10545|-|2075230.320892334 +6220|-|2074341.7611694336 +8616|-|2072887.6767425537 +5230|-|2072161.7690200806 +13916|-|2070504.7272949219 +4299|-|2069922.978149414 +894|-|2069688.126586914 +17847|-|2063367.09375 +18879|-|2061902.25 +13036|-|2061600.186340332 +10606|-|2060492.3840332031 +9454|-|2060016.404296875 +118|-|2059808.7673339844 +9601|-|2059715.8142089844 +13769|-|2057667.9653320312 +1987|-|2057289.2335205078 +13863|-|2055367.972869873 +13562|-|2054754.173828125 +1840|-|2054183.9311523438 +17995|-|2053221.9276733398 +17389|-|2051128.1726074219 +15168|-|2045987.4976501465 +2139|-|2045365.2990722656 +4024|-|2044243.0604248047 +8964|-|2041648.8778686523 +181|-|2040167.0458984375 +7628|-|2039548.974243164 +3|-|2038846.1099853516 +15553|-|2036959.0168457031 +11355|-|2035405.6726074219 +13006|-|2034991.224761963 +3091|-|2031393.5194702148 +1281|-|2030628.4670410156 +1408|-|2028621.6350097656 +18211|-|2024538.7075805664 +2287|-|2020754.2392578125 +6228|-|2019198.8252868652 +4362|-|2018495.297241211 +10873|-|2013280.2802734375 +7383|-|2009581.8969726562 +1386|-|2006544.2039794922 +9820|-|2005815.8198242188 +18134|-|2003409.7178649902 +15727|-|2000654.5 +157|-|2000148.1732177734 +19571|-|1999891.1595153809 +17728|-|1997944.4525146484 +5278|-|1996644.1983642578 +17737|-|1994653.74609375 +10220|-|1989890.9174804688 +1397|-|1984509.1595458984 +6195|-|1983928.1798095703 +4270|-|1983726.9793701172 +16965|-|1983286.1907958984 +1683|-|1980638.5913085938 +13086|-|1978609.3798828125 +7124|-|1974039.398071289 +5211|-|1973843.7998046875 +6794|-|1973149.444885254 +257|-|1973035.476928711 +6995|-|1968281.5246582031 +8447|-|1967292.732269287 +15873|-|1967257.8836975098 +12862|-|1964014.1873779297 +8295|-|1961467.1154785156 +931|-|1958825.215057373 +6876|-|1957359.5209960938 +1932|-|1954592.3408203125 +1061|-|1952688.0393676758 +18108|-|1951143.7312927246 +5138|-|1950861.0397338867 +12598|-|1950211.6165161133 +10829|-|1943924.6043701172 +11950|-|1941210.9121704102 +12076|-|1939324.0358276367 +2176|-|1938691.3228149414 +6616|-|1937401.9128417969 +5893|-|1934358.6018066406 +976|-|1933066.8347167969 +13173|-|1932557.5862121582 +14947|-|1929229.9713745117 +16857|-|1928814.7243652344 +13403|-|1928702.8103027344 +4819|-|1926969.7286376953 +13127|-|1926929.848815918 +6871|-|1926787.6062011719 +15465|-|1925145.1381530762 +1131|-|1920005.5 +11845|-|1913576.3525390625 +8364|-|1909122.1602172852 +16588|-|1904272.3926391602 +6759|-|1903906.2498779297 +11586|-|1901895.5828857422 +8145|-|1901787.6020507812 +17333|-|1897297.1284179688 +13290|-|1890633.75 +6499|-|1887620.9167480469 +4881|-|1887535.8764038086 +7147|-|1886710.140991211 +3883|-|1886567.746307373 +18911|-|1885597.1535644531 +11336|-|1883573.6238708496 +8653|-|1883275.7177581787 +19476|-|1881492.5407104492 +14799|-|1880543.4484863281 +14491|-|1879219.9541625977 +11815|-|1877434.3993530273 +3173|-|1874302.1049499512 +7161|-|1873023.4817504883 +14631|-|1873015.319519043 +4247|-|1869912.9061279297 +3568|-|1865824.3737792969 +1500|-|1865450.9802246094 +11833|-|1863665.2496948242 +495|-|1860771.3789367676 +6776|-|1855589.0936889648 +11374|-|1855221.1085205078 +5637|-|1853782.114654541 +3597|-|1852826.880432129 +981|-|1852083.6515808105 +16076|-|1850349.7155761719 +17597|-|1845421.0075378418 +19609|-|1843185.4454956055 +10997|-|1843072.0914916992 +3403|-|1842975.0 +897|-|1842845.052520752 +16697|-|1840630.7087402344 +17644|-|1840597.7307128906 +6485|-|1838812.0330810547 +5492|-|1836202.818359375 +12038|-|1835075.0694580078 +9325|-|1832634.8212890625 +10637|-|1832347.4095458984 +11318|-|1830158.4047088623 +4357|-|1828729.9438476562 +18553|-|1826335.2055358887 +12623|-|1825950.817199707 +961|-|1825869.6166992188 +1677|-|1821816.9250488281 +8211|-|1820432.5127563477 +19719|-|1819333.578491211 +19663|-|1819074.4201660156 +16296|-|1818353.7156066895 +16527|-|1817834.3540039062 +4964|-|1815400.0594177246 +1769|-|1812929.1693115234 +13126|-|1808799.9389648438 +7854|-|1807608.1387939453 +18380|-|1803641.2076416016 +6584|-|1802346.9595947266 +7665|-|1801765.3981933594 +16553|-|1796146.7329101562 +17761|-|1795095.779296875 +11179|-|1794890.2955627441 +15171|-|1794148.6752929688 +3018|-|1793183.8253173828 +15741|-|1788611.9757080078 +5331|-|1783901.2844848633 +9860|-|1775071.2819213867 +7984|-|1774302.801361084 +15354|-|1774270.7454833984 +17884|-|1774212.3813476562 +16257|-|1771869.7191467285 +10696|-|1768645.2502441406 +2104|-|1767902.6162109375 +14465|-|1764946.429901123 +10089|-|1764692.337890625 +6719|-|1762699.5961914062 +3648|-|1760594.3675842285 +7241|-|1759913.6359558105 +11122|-|1757429.9960289001 +17019|-|1752560.6893920898 +13877|-|1744271.0842895508 +15325|-|1743826.3114013672 +17860|-|1739870.3830566406 +2236|-|1739795.7482910156 +4436|-|1738760.310546875 +7701|-|1738670.4733886719 +8147|-|1736855.1896209717 +6676|-|1736341.4271240234 +19505|-|1735413.412902832 +9885|-|1731366.2120361328 +2112|-|1725934.1193237305 +5330|-|1722196.964630127 +3561|-|1720377.9982910156 +10104|-|1714419.1653289795 +16362|-|1712457.3900756836 +15573|-|1712365.4072265625 +15006|-|1711381.300994873 +14629|-|1709942.0542907715 +9612|-|1709528.3438720703 +19910|-|1709211.0869750977 +13145|-|1708907.3772583008 +11494|-|1707973.7124023438 +15895|-|1706999.4177246094 +8239|-|1705479.1485595703 +2403|-|1705331.1588134766 +19436|-|1702706.0302734375 +3476|-|1702335.8241577148 +6828|-|1702292.1298828125 +771|-|1701589.5 +8448|-|1700312.4758300781 +3755|-|1699047.086517334 +13895|-|1698679.0136871338 +9785|-|1698056.3375549316 +6180|-|1695571.4729003906 +532|-|1694356.161058545 +6741|-|1692552.4584350586 +19964|-|1692367.5817871094 +3747|-|1691244.5666503906 +3253|-|1690719.3782958984 +16119|-|1688339.222946167 +7113|-|1681910.9939575195 +12368|-|1681219.8216552734 +16378|-|1679705.6228027344 +1393|-|1675545.301147461 +11119|-|1675453.435546875 +4469|-|1674023.5486450195 +6955|-|1672618.8428955078 +11579|-|1672345.3000488281 +19898|-|1671781.7362670898 +15351|-|1659204.3444824219 +6133|-|1658215.4739990234 +9110|-|1658054.6356811523 +2979|-|1656016.7265930176 +18764|-|1653708.4138183594 +8995|-|1653627.604309082 +13096|-|1651408.7465515137 +15062|-|1650548.0698242188 +7924|-|1650202.4209136963 +10076|-|1647970.1915283203 +15859|-|1646036.3041992188 +17932|-|1642640.6559143066 +19694|-|1642089.563293457 +13827|-|1642001.3620605469 +17963|-|1639689.0 +10698|-|1635848.2767028809 +18003|-|1633530.8166503906 +8416|-|1633366.7739562988 +476|-|1631154.0655975342 +2806|-|1630782.7474975586 +12129|-|1628615.5079345703 +11215|-|1626624.6862792969 +14061|-|1624933.5075683594 +5956|-|1623586.1042785645 +9043|-|1622670.3633728027 +13287|-|1621980.3876342773 +11410|-|1621420.8666992188 +13990|-|1621268.2245483398 +12952|-|1619215.2041625977 +15181|-|1619088.6878051758 +9784|-|1618120.4653015137 +10733|-|1616168.8916931152 +16054|-|1614531.2112426758 +5864|-|1614397.8735351562 +1875|-|1611927.0240783691 +17381|-|1611664.8427734375 +14562|-|1607467.8866577148 +575|-|1605941.6989746094 +2005|-|1605591.665222168 +4332|-|1605448.8093566895 +4653|-|1602596.349975586 +15403|-|1601830.3666992188 +17430|-|1599681.3586425781 +4798|-|1593630.5130004883 +12991|-|1593321.5407714844 +15653|-|1593138.7158660889 +10066|-|1593049.0897216797 +8892|-|1592100.8681030273 +6708|-|1590159.0702438354 +9825|-|1589403.9376831055 +8271|-|1588475.4641113281 +17084|-|1584280.8524169922 +4003|-|1583631.0 +869|-|1582643.133178711 +16400|-|1582313.2043457031 +19088|-|1581708.5712890625 +6581|-|1581346.8233642578 +9481|-|1581048.6053466797 +6092|-|1580846.4543457031 +3624|-|1578777.2731323242 +6503|-|1578507.8547363281 +14557|-|1578281.0028686523 +2428|-|1577543.8458251953 +15513|-|1573560.174041748 +4641|-|1573363.567993164 +10152|-|1570213.544921875 +5932|-|1566902.4616699219 +7482|-|1561323.4680175781 +13745|-|1558358.3333435059 +2251|-|1558274.6385498047 +9845|-|1558068.0759277344 +7603|-|1557388.1533813477 +1809|-|1553837.161743164 +18128|-|1547643.3653259277 +8086|-|1543199.0756835938 +14948|-|1541721.517715454 +16725|-|1540948.5 +2999|-|1540317.6873779297 +8861|-|1540008.5061340332 +1964|-|1538815.2198791504 +19374|-|1537884.8074035645 +15428|-|1535994.3779907227 +7449|-|1534782.4838256836 +16884|-|1534509.1881103516 +10271|-|1534397.2959594727 +11782|-|1529963.159790039 +8184|-|1529750.6717834473 +4560|-|1527433.2497253418 +4616|-|1525374.4865722656 +3814|-|1524076.9855957031 +17265|-|1523932.0370178223 +16520|-|1522906.3503417969 +10475|-|1518705.0485229492 +5094|-|1517317.8380126953 +8626|-|1515142.0741577148 +19895|-|1512286.6967773438 +19933|-|1506235.3649597168 +6854|-|1505626.0 +13995|-|1505562.1680908203 +7102|-|1504945.6089172363 +9079|-|1501237.1293945312 +18329|-|1500146.969116211 +3742|-|1496990.726196289 +12395|-|1496904.4160766602 +12214|-|1496489.390258789 +12298|-|1495554.3113708496 +4978|-|1495389.471130371 +2927|-|1494280.0918579102 +2119|-|1494151.127380371 +15143|-|1492039.783859253 +14548|-|1487406.5622558594 +840|-|1486128.9608764648 +5902|-|1486097.3247070312 +10614|-|1482144.7607421875 +5895|-|1481356.763305664 +15958|-|1480951.6381835938 +11408|-|1479948.9807128906 +8407|-|1474236.0 +6243|-|1471007.8878173828 +10389|-|1469004.5239562988 +13871|-|1468938.6767578125 +19811|-|1464597.0150146484 +10495|-|1464290.519104004 +4389|-|1463010.7704467773 +1311|-|1461703.4008789062 +17874|-|1459408.8374023438 +6597|-|1458761.8845214844 +19211|-|1456741.605255127 +12879|-|1456178.2114257812 +8840|-|1455731.4017944336 +14755|-|1454890.5603027344 +16957|-|1454465.9743652344 +9257|-|1454388.7916259766 +5193|-|1454011.3244018555 +6884|-|1452474.5421142578 +19948|-|1452024.0 +15076|-|1448395.0119018555 +16016|-|1447557.4293823242 +11693|-|1445839.7114868164 +6975|-|1440516.9924316406 +4290|-|1439768.5415039062 +18900|-|1438722.1258544922 +14383|-|1438477.8793945312 +15098|-|1435941.8385314941 +9322|-|1435282.8387451172 +458|-|1433040.412902832 +10042|-|1432906.392364502 +5052|-|1431900.9575500488 +6600|-|1431116.5880126953 +3630|-|1428665.0668945312 +9636|-|1428193.8202514648 +16511|-|1427308.7228393555 +4045|-|1427248.2903442383 +19562|-|1426348.7884521484 +8814|-|1425690.0772705078 +2616|-|1425178.0203857422 +4587|-|1425109.3546142578 +148|-|1424237.3502502441 +2712|-|1423780.2355957031 +10863|-|1423386.1506347656 +16096|-|1421942.0442504883 +18936|-|1421938.6294555664 +18327|-|1419872.9162597656 +11620|-|1419050.1036071777 +3740|-|1418609.87789917 +3457|-|1418603.5 +1185|-|1417637.4230957031 +8178|-|1417357.3143310547 +17791|-|1413293.0977172852 +13608|-|1411323.150390625 +17849|-|1409613.478088379 +6814|-|1406228.4159851074 +14022|-|1406138.0842285156 +14231|-|1403771.5576171875 +19546|-|1402854.5819702148 +19619|-|1402389.2103271484 +5609|-|1402302.5663909912 +5342|-|1401567.6424560547 +3084|-|1401096.0903930664 +5708|-|1400334.841003418 +17998|-|1399862.4307250977 +19850|-|1397630.3699951172 +14004|-|1395443.0749053955 +13071|-|1394653.2860870361 +2797|-|1393747.5387573242 +2866|-|1392947.2953796387 +19809|-|1389067.7252197266 +13600|-|1380865.775756836 +13614|-|1380654.2938079834 +5884|-|1380319.7587280273 +9404|-|1378623.6877441406 +10656|-|1376954.3553466797 +12324|-|1376502.3815917969 +7325|-|1375030.3835601807 +13295|-|1373987.3495178223 +11864|-|1373555.7197265625 +6987|-|1373481.4578399658 +8386|-|1371854.3801879883 +10916|-|1370374.3459472656 +12867|-|1369058.1170043945 +14668|-|1369040.3050231934 +13383|-|1367342.254119873 +18572|-|1366954.013305664 +1152|-|1366861.3978271484 +6015|-|1366452.2076416016 +3344|-|1366185.1764526367 +7889|-|1365521.9282226562 +13345|-|1364088.9291992188 +6276|-|1363421.6049804688 +8069|-|1361824.2585449219 +17509|-|1360892.4440917969 +15137|-|1358678.0104980469 +17163|-|1357391.5466308594 +4704|-|1356692.3569335938 +8609|-|1356578.2471618652 +12644|-|1356088.1536560059 +17141|-|1356022.3646240234 +11805|-|1354826.7747039795 +6386|-|1354187.1807861328 +3004|-|1352173.4365234375 +8634|-|1350211.761291504 +4399|-|1349881.159790039 +10362|-|1349411.3036499023 +1572|-|1348835.2299804688 +7359|-|1348224.1096496582 +11884|-|1346696.8058624268 +11671|-|1346424.1040039062 +5350|-|1346359.3221435547 +3119|-|1345996.4392089844 +5307|-|1345355.947265625 +16117|-|1345045.181640625 +8715|-|1342665.7254638672 +5398|-|1341179.2985229492 +7627|-|1338820.5560455322 +8457|-|1337714.7106933594 +4958|-|1334732.7673339844 +84|-|1334146.7134857178 +6932|-|1333235.3314208984 +757|-|1332921.0741577148 +4076|-|1332441.0382080078 +1751|-|1329112.306640625 +15701|-|1327052.1989135742 +4119|-|1326549.874847412 +1562|-|1325604.3093261719 +8741|-|1325517.5654296875 +1135|-|1325422.705871582 +1002|-|1323418.641418457 +5832|-|1323085.6803588867 +5368|-|1322793.980255127 +5382|-|1322628.8184814453 +5616|-|1319082.2923583984 +2832|-|1318691.9218444824 +3895|-|1317858.4164886475 +8629|-|1317756.5368652344 +5709|-|1317058.6618652344 +18383|-|1316451.0198669434 +15797|-|1314806.6162109375 +1900|-|1313660.433959961 +13882|-|1310455.8077392578 +6785|-|1309877.7795410156 +14855|-|1309280.8302001953 +7761|-|1308602.2880859375 +14268|-|1306810.3637695312 +6257|-|1306056.9331054688 +19002|-|1305509.5361328125 +5095|-|1303729.0596923828 +10320|-|1301657.6178588867 +7826|-|1299561.6943359375 +13359|-|1298717.173675537 +7436|-|1298127.3671875 +5644|-|1295055.8004760742 +11327|-|1290526.4305114746 +5277|-|1289329.6742248535 +15932|-|1286235.8208007812 +14322|-|1284809.348083496 +144|-|1284270.1287841797 +3043|-|1281162.7379608154 +16788|-|1280955.3276824951 +17136|-|1280443.0877685547 +12560|-|1279117.9238586426 +13833|-|1278834.75 +5414|-|1277893.2998199463 +12582|-|1277592.3092651367 +4644|-|1277535.0036621094 +14032|-|1277077.8298950195 +18325|-|1271719.6611328125 +7072|-|1271228.4482421875 +16868|-|1267469.430847168 +8137|-|1267425.8177490234 +5976|-|1266206.9095458984 +14125|-|1265569.0479125977 +13299|-|1265287.6037597656 +18376|-|1264249.3371582031 +6157|-|1261759.8669433594 +5002|-|1261669.6770019531 +13368|-|1260918.5923461914 +15589|-|1260059.7495117188 +2149|-|1258981.3828125 +9639|-|1256283.3509521484 +11689|-|1256027.9272460938 +9083|-|1245924.1743164062 +16231|-|1242625.6755065918 +5084|-|1242385.3291015625 +11634|-|1240760.2196655273 +15617|-|1239731.25 +9865|-|1237181.5987701416 +14212|-|1236365.53515625 +10325|-|1235223.3427734375 +19582|-|1235105.7646484375 +740|-|1234746.8372955322 +19231|-|1233623.062133789 +16840|-|1233063.890197754 +5703|-|1231744.3662261963 +5761|-|1229435.17578125 +15630|-|1226611.6397094727 +10408|-|1224698.3697509766 +9177|-|1221942.5216674805 +13389|-|1221666.7762756348 +6104|-|1221577.9013671875 +9673|-|1218826.6256866455 +2707|-|1217124.4658203125 +18672|-|1214208.7954711914 +5112|-|1209590.2445220947 +6264|-|1208318.5409545898 +18496|-|1207881.75 +10971|-|1207183.5093688965 +19059|-|1206729.8780822754 +431|-|1205938.4302368164 +3821|-|1201192.758178711 +826|-|1200454.5931091309 +3317|-|1200440.8734436035 +19689|-|1198899.5228881836 +19641|-|1198797.9747314453 +6379|-|1197195.5 +814|-|1194417.4224243164 +18643|-|1194000.783569336 +11865|-|1193965.7729492188 +12393|-|1193896.8090820312 +9218|-|1193660.5513916016 +8674|-|1191881.3533477783 +8582|-|1191803.997253418 +13084|-|1191508.0 +18844|-|1190239.9542236328 +16061|-|1189935.0 +6134|-|1185550.8386230469 +8628|-|1183245.5668945312 +8884|-|1181547.4489746094 +7697|-|1181032.5 +9044|-|1180922.6353149414 +13257|-|1180158.620300293 +8066|-|1178808.1107330322 +5876|-|1177376.77734375 +14694|-|1177059.351852417 +16062|-|1175391.0173034668 +9104|-|1175178.9472961426 +11600|-|1175091.096496582 +10337|-|1172684.9451904297 +19188|-|1172349.7580566406 +8833|-|1171372.904724121 +6895|-|1170602.089416504 +14100|-|1168878.41796875 +13538|-|1168554.3215332031 +3408|-|1166645.1654052734 +1860|-|1165673.7287902832 +13436|-|1164278.6790161133 +19325|-|1162733.705291748 +7403|-|1161982.029724121 +4882|-|1161404.8461914062 +13105|-|1161320.5768127441 +17880|-|1161256.0773925781 +19284|-|1160927.5817565918 +13476|-|1159035.1672973633 +18913|-|1158208.258392334 +18523|-|1158135.0402832031 +12508|-|1157538.5005187988 +9090|-|1156362.6936035156 +17653|-|1154338.0590820312 +3926|-|1152652.4869995117 +10183|-|1148324.5918579102 +7556|-|1146268.1630859375 +16436|-|1142656.5260009766 +4741|-|1141614.0197753906 +15651|-|1141497.958190918 +3183|-|1140081.3345947266 +9532|-|1139902.5 +16403|-|1139306.3049316406 +2368|-|1137421.1923828125 +3889|-|1136395.5505371094 +2885|-|1135838.1436157227 +7851|-|1135110.7958374023 +16234|-|1135017.2622680664 +12746|-|1134531.0544433594 +2647|-|1132941.1064453125 +5373|-|1132158.0131225586 +10340|-|1132004.2142028809 +8873|-|1131949.2911987305 +1132|-|1131338.8703613281 +15594|-|1131328.5715942383 +4376|-|1130282.1826171875 +240|-|1126682.5329589844 +2231|-|1124447.1466827393 +929|-|1121383.9423828125 +11599|-|1119307.2609481812 +3765|-|1119093.4799194336 +17635|-|1118420.1767578125 +7119|-|1118285.0696411133 +15121|-|1117715.336013794 +11858|-|1116963.5603027344 +16963|-|1116929.478881836 +16356|-|1113649.0235595703 +6924|-|1112198.37890625 +16223|-|1111257.0 +18091|-|1110043.0338134766 +12628|-|1108954.82321167 +16043|-|1108831.0647735596 +9402|-|1108290.4839019775 +708|-|1107084.0362548828 +4078|-|1105993.9275054932 +17593|-|1104713.4141845703 +12776|-|1104362.6319885254 +7583|-|1102813.4978027344 +14619|-|1102675.7803344727 +8842|-|1100110.2694702148 +4196|-|1099726.5893554688 +2019|-|1098178.6320495605 +6863|-|1097246.3224487305 +6489|-|1096503.0916137695 +2459|-|1094813.0198364258 +11964|-|1094484.9845581055 +3236|-|1093969.8435058594 +17647|-|1093809.1983642578 +17648|-|1093114.6135253906 +119|-|1092687.5137939453 +9626|-|1092079.9865722656 +9124|-|1091569.6478271484 +13175|-|1089851.7216796875 +2532|-|1088706.334411621 +16083|-|1088295.4176940918 +8874|-|1086011.3367004395 +12872|-|1082970.2655029297 +19821|-|1082520.8139038086 +4800|-|1080389.6745300293 +18696|-|1079685.401550293 +19545|-|1079184.3186950684 +13120|-|1077742.2612304688 +10588|-|1076203.8264770508 +17696|-|1075092.6732177734 +14651|-|1073222.2071533203 +903|-|1071146.7752304077 +5858|-|1070259.463256836 +8302|-|1069504.8403320312 +18728|-|1069225.5051574707 +18026|-|1068569.0145874023 +19383|-|1066907.5825958252 +18690|-|1065930.9407958984 +5924|-|1065143.144104004 +4880|-|1065011.75 +12439|-|1064381.166442871 +16529|-|1062371.728088379 +19653|-|1057683.5363769531 +3136|-|1056810.4391860962 +18932|-|1056193.6198425293 +2124|-|1054160.5241699219 +16851|-|1052646.814819336 +10123|-|1051623.9782714844 +5618|-|1048447.9822540283 +19851|-|1045187.8079223633 +16278|-|1044808.3540039062 +11479|-|1044276.2375640869 +13263|-|1042046.1585388184 +6041|-|1041123.3318481445 +7193|-|1040455.2977294922 +19408|-|1039430.0012512207 +11260|-|1036828.5228881836 +5179|-|1035633.4487304688 +1331|-|1034398.0 +7706|-|1034249.3519592285 +8436|-|1033549.3817138672 +1801|-|1031886.0131835938 +4170|-|1031642.9565429688 +11827|-|1031139.4237976074 +17114|-|1027985.8488464355 +18278|-|1026583.1158447266 +1995|-|1025165.6577148438 +7667|-|1022980.114654541 +6559|-|1021635.4828643799 +17488|-|1021612.1164245605 +16059|-|1019781.1790771484 +7633|-|1018782.5820922852 +10032|-|1016809.5 +2899|-|1016438.739440918 +14628|-|1016033.214302063 +10126|-|1015846.7950439453 +3884|-|1014413.5345458984 +16913|-|1013604.354309082 +18644|-|1010288.1134033203 +19870|-|1007919.3837890625 +18564|-|1007416.2255859375 +10179|-|1004920.0225830078 +883|-|1004650.6300048828 +3627|-|1004461.0607299805 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out new file mode 100644 index 000000000000..99541acd0f54 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q12.out @@ -0,0 +1,3 @@ +2 +MAIL|-|647|-|945 +SHIP|-|620|-|943 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out new file mode 100644 index 000000000000..ae4362f61da3 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q13.out @@ -0,0 +1,38 @@ +37 +0|-|5000 +10|-|665 +9|-|657 +11|-|621 +12|-|567 +8|-|564 +13|-|492 +18|-|482 +7|-|480 +20|-|456 +14|-|456 +16|-|449 +19|-|447 +15|-|432 +17|-|423 +21|-|412 +22|-|371 +6|-|337 +23|-|323 +24|-|256 +25|-|204 +5|-|204 +26|-|155 +27|-|141 +28|-|97 +4|-|94 +29|-|64 +3|-|48 +30|-|27 +31|-|26 +32|-|14 +33|-|11 +2|-|11 +34|-|6 +35|-|5 +1|-|2 +36|-|1 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out new file mode 100644 index 000000000000..f8627a6705e2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q14.out @@ -0,0 +1,2 @@ +1 +16.283855689005982 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out new file mode 100644 index 000000000000..9dac6c899883 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q15.out @@ -0,0 +1,2 @@ +1 +677|-|Supplier#000000677|-|8mhrffG7D2WJBSQbOGstQ|-|23-290-639-3315|-|1614410.2928000002 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out new file mode 100644 index 000000000000..341afcb5718b --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q16.out @@ -0,0 +1,2763 @@ +2762 +Brand#14|-|SMALL ANODIZED NICKEL|-|45|-|12 +Brand#22|-|SMALL BURNISHED BRASS|-|19|-|12 +Brand#25|-|PROMO POLISHED COPPER|-|14|-|12 +Brand#35|-|LARGE ANODIZED STEEL|-|45|-|12 +Brand#35|-|PROMO BRUSHED COPPER|-|9|-|12 +Brand#51|-|ECONOMY ANODIZED STEEL|-|9|-|12 +Brand#53|-|LARGE BRUSHED NICKEL|-|45|-|12 +Brand#11|-|ECONOMY POLISHED COPPER|-|14|-|8 +Brand#11|-|LARGE PLATED STEEL|-|23|-|8 +Brand#11|-|PROMO POLISHED STEEL|-|23|-|8 +Brand#11|-|STANDARD ANODIZED COPPER|-|9|-|8 +Brand#12|-|ECONOMY BURNISHED BRASS|-|9|-|8 +Brand#12|-|LARGE ANODIZED BRASS|-|14|-|8 +Brand#12|-|SMALL ANODIZED TIN|-|23|-|8 +Brand#12|-|SMALL BRUSHED NICKEL|-|23|-|8 +Brand#12|-|STANDARD ANODIZED BRASS|-|3|-|8 +Brand#12|-|STANDARD BURNISHED TIN|-|23|-|8 +Brand#13|-|ECONOMY POLISHED BRASS|-|9|-|8 +Brand#13|-|LARGE BURNISHED COPPER|-|45|-|8 +Brand#13|-|MEDIUM ANODIZED STEEL|-|23|-|8 +Brand#13|-|MEDIUM PLATED NICKEL|-|3|-|8 +Brand#13|-|PROMO BURNISHED BRASS|-|9|-|8 +Brand#13|-|PROMO POLISHED BRASS|-|3|-|8 +Brand#13|-|PROMO POLISHED TIN|-|36|-|8 +Brand#13|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#13|-|STANDARD BRUSHED STEEL|-|9|-|8 +Brand#14|-|ECONOMY BRUSHED TIN|-|3|-|8 +Brand#14|-|ECONOMY BURNISHED TIN|-|23|-|8 +Brand#14|-|PROMO BRUSHED STEEL|-|9|-|8 +Brand#14|-|PROMO PLATED TIN|-|45|-|8 +Brand#15|-|ECONOMY PLATED TIN|-|9|-|8 +Brand#15|-|STANDARD BRUSHED COPPER|-|14|-|8 +Brand#15|-|STANDARD PLATED TIN|-|3|-|8 +Brand#21|-|ECONOMY POLISHED TIN|-|3|-|8 +Brand#21|-|PROMO POLISHED COPPER|-|9|-|8 +Brand#21|-|PROMO POLISHED TIN|-|49|-|8 +Brand#21|-|SMALL POLISHED STEEL|-|3|-|8 +Brand#21|-|STANDARD PLATED BRASS|-|49|-|8 +Brand#21|-|STANDARD PLATED NICKEL|-|49|-|8 +Brand#22|-|ECONOMY ANODIZED TIN|-|49|-|8 +Brand#22|-|ECONOMY BRUSHED BRASS|-|14|-|8 +Brand#22|-|LARGE BURNISHED TIN|-|36|-|8 +Brand#22|-|MEDIUM ANODIZED STEEL|-|36|-|8 +Brand#22|-|MEDIUM PLATED STEEL|-|9|-|8 +Brand#22|-|PROMO POLISHED NICKEL|-|9|-|8 +Brand#22|-|SMALL ANODIZED STEEL|-|19|-|8 +Brand#22|-|STANDARD ANODIZED COPPER|-|23|-|8 +Brand#23|-|ECONOMY BRUSHED NICKEL|-|23|-|8 +Brand#23|-|LARGE ANODIZED BRASS|-|9|-|8 +Brand#23|-|LARGE ANODIZED STEEL|-|23|-|8 +Brand#23|-|SMALL BRUSHED COPPER|-|23|-|8 +Brand#23|-|STANDARD BRUSHED TIN|-|3|-|8 +Brand#23|-|STANDARD BURNISHED NICKEL|-|49|-|8 +Brand#23|-|STANDARD PLATED NICKEL|-|36|-|8 +Brand#24|-|ECONOMY ANODIZED BRASS|-|19|-|8 +Brand#24|-|ECONOMY POLISHED BRASS|-|36|-|8 +Brand#24|-|LARGE BURNISHED STEEL|-|14|-|8 +Brand#24|-|MEDIUM PLATED NICKEL|-|36|-|8 +Brand#25|-|ECONOMY BRUSHED STEEL|-|49|-|8 +Brand#25|-|MEDIUM BURNISHED TIN|-|3|-|8 +Brand#25|-|PROMO ANODIZED TIN|-|36|-|8 +Brand#25|-|PROMO PLATED NICKEL|-|3|-|8 +Brand#25|-|SMALL BURNISHED BRASS|-|3|-|8 +Brand#31|-|LARGE ANODIZED BRASS|-|3|-|8 +Brand#31|-|SMALL ANODIZED COPPER|-|3|-|8 +Brand#31|-|SMALL ANODIZED NICKEL|-|9|-|8 +Brand#31|-|SMALL ANODIZED STEEL|-|14|-|8 +Brand#32|-|MEDIUM ANODIZED STEEL|-|49|-|8 +Brand#32|-|MEDIUM BURNISHED COPPER|-|19|-|8 +Brand#32|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#32|-|STANDARD BURNISHED STEEL|-|45|-|8 +Brand#34|-|ECONOMY ANODIZED NICKEL|-|49|-|8 +Brand#34|-|LARGE BURNISHED TIN|-|49|-|8 +Brand#34|-|MEDIUM BURNISHED NICKEL|-|3|-|8 +Brand#34|-|PROMO ANODIZED TIN|-|3|-|8 +Brand#34|-|SMALL BRUSHED TIN|-|3|-|8 +Brand#34|-|STANDARD BURNISHED TIN|-|23|-|8 +Brand#35|-|MEDIUM BRUSHED STEEL|-|45|-|8 +Brand#35|-|PROMO BURNISHED STEEL|-|14|-|8 +Brand#35|-|SMALL BURNISHED STEEL|-|23|-|8 +Brand#35|-|SMALL POLISHED COPPER|-|14|-|8 +Brand#35|-|STANDARD PLATED COPPER|-|9|-|8 +Brand#41|-|ECONOMY BRUSHED BRASS|-|23|-|8 +Brand#41|-|LARGE BURNISHED STEEL|-|23|-|8 +Brand#41|-|PROMO BURNISHED TIN|-|14|-|8 +Brand#41|-|PROMO PLATED STEEL|-|36|-|8 +Brand#41|-|PROMO POLISHED TIN|-|19|-|8 +Brand#41|-|SMALL BURNISHED COPPER|-|23|-|8 +Brand#42|-|LARGE POLISHED TIN|-|14|-|8 +Brand#42|-|MEDIUM ANODIZED TIN|-|49|-|8 +Brand#42|-|MEDIUM BRUSHED TIN|-|14|-|8 +Brand#42|-|MEDIUM BURNISHED NICKEL|-|23|-|8 +Brand#42|-|MEDIUM PLATED COPPER|-|45|-|8 +Brand#42|-|MEDIUM PLATED TIN|-|45|-|8 +Brand#42|-|SMALL PLATED COPPER|-|36|-|8 +Brand#43|-|ECONOMY BRUSHED STEEL|-|45|-|8 +Brand#43|-|LARGE BRUSHED COPPER|-|19|-|8 +Brand#43|-|PROMO BRUSHED BRASS|-|36|-|8 +Brand#43|-|SMALL BURNISHED TIN|-|45|-|8 +Brand#43|-|SMALL PLATED COPPER|-|45|-|8 +Brand#44|-|PROMO POLISHED TIN|-|23|-|8 +Brand#44|-|SMALL POLISHED NICKEL|-|14|-|8 +Brand#44|-|SMALL POLISHED TIN|-|45|-|8 +Brand#44|-|STANDARD BURNISHED COPPER|-|3|-|8 +Brand#51|-|LARGE ANODIZED BRASS|-|19|-|8 +Brand#51|-|LARGE POLISHED COPPER|-|23|-|8 +Brand#51|-|MEDIUM ANODIZED TIN|-|9|-|8 +Brand#51|-|MEDIUM ANODIZED TIN|-|14|-|8 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|23|-|8 +Brand#51|-|SMALL ANODIZED COPPER|-|45|-|8 +Brand#51|-|SMALL ANODIZED COPPER|-|49|-|8 +Brand#51|-|SMALL BRUSHED COPPER|-|45|-|8 +Brand#51|-|SMALL BRUSHED TIN|-|36|-|8 +Brand#51|-|STANDARD POLISHED TIN|-|3|-|8 +Brand#52|-|ECONOMY ANODIZED STEEL|-|3|-|8 +Brand#52|-|ECONOMY PLATED TIN|-|19|-|8 +Brand#52|-|LARGE PLATED TIN|-|3|-|8 +Brand#52|-|MEDIUM ANODIZED TIN|-|19|-|8 +Brand#52|-|MEDIUM BURNISHED COPPER|-|3|-|8 +Brand#52|-|PROMO POLISHED BRASS|-|23|-|8 +Brand#52|-|SMALL PLATED COPPER|-|36|-|8 +Brand#52|-|SMALL POLISHED NICKEL|-|9|-|8 +Brand#52|-|STANDARD POLISHED NICKEL|-|45|-|8 +Brand#53|-|ECONOMY POLISHED STEEL|-|45|-|8 +Brand#53|-|LARGE POLISHED NICKEL|-|3|-|8 +Brand#53|-|SMALL BRUSHED COPPER|-|14|-|8 +Brand#53|-|STANDARD PLATED STEEL|-|45|-|8 +Brand#54|-|ECONOMY POLISHED BRASS|-|49|-|8 +Brand#54|-|ECONOMY POLISHED TIN|-|23|-|8 +Brand#54|-|LARGE ANODIZED NICKEL|-|49|-|8 +Brand#54|-|MEDIUM BRUSHED STEEL|-|9|-|8 +Brand#54|-|SMALL BURNISHED NICKEL|-|14|-|8 +Brand#54|-|SMALL PLATED TIN|-|14|-|8 +Brand#54|-|STANDARD BURNISHED STEEL|-|14|-|8 +Brand#54|-|STANDARD PLATED BRASS|-|23|-|8 +Brand#55|-|MEDIUM BURNISHED TIN|-|36|-|8 +Brand#55|-|PROMO ANODIZED BRASS|-|14|-|8 +Brand#55|-|STANDARD BURNISHED COPPER|-|45|-|8 +Brand#15|-|STANDARD PLATED TIN|-|36|-|7 +Brand#23|-|SMALL POLISHED BRASS|-|49|-|7 +Brand#42|-|STANDARD PLATED COPPER|-|19|-|7 +Brand#51|-|LARGE POLISHED NICKEL|-|14|-|7 +Brand#11|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#11|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#11|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#11|-|ECONOMY BRUSHED COPPER|-|3|-|4 +Brand#11|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#11|-|ECONOMY BRUSHED STEEL|-|9|-|4 +Brand#11|-|ECONOMY BRUSHED STEEL|-|36|-|4 +Brand#11|-|ECONOMY BURNISHED BRASS|-|36|-|4 +Brand#11|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#11|-|ECONOMY BURNISHED COPPER|-|49|-|4 +Brand#11|-|ECONOMY BURNISHED NICKEL|-|14|-|4 +Brand#11|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#11|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#11|-|ECONOMY PLATED NICKEL|-|45|-|4 +Brand#11|-|ECONOMY PLATED TIN|-|9|-|4 +Brand#11|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#11|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#11|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#11|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#11|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#11|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#11|-|LARGE ANODIZED COPPER|-|23|-|4 +Brand#11|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#11|-|LARGE ANODIZED STEEL|-|9|-|4 +Brand#11|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#11|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#11|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#11|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#11|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#11|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#11|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#11|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#11|-|LARGE PLATED COPPER|-|23|-|4 +Brand#11|-|LARGE PLATED TIN|-|9|-|4 +Brand#11|-|LARGE PLATED TIN|-|14|-|4 +Brand#11|-|LARGE PLATED TIN|-|23|-|4 +Brand#11|-|LARGE POLISHED NICKEL|-|49|-|4 +Brand#11|-|MEDIUM ANODIZED BRASS|-|45|-|4 +Brand#11|-|MEDIUM ANODIZED TIN|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED BRASS|-|45|-|4 +Brand#11|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#11|-|MEDIUM BRUSHED NICKEL|-|36|-|4 +Brand#11|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#11|-|MEDIUM BURNISHED COPPER|-|9|-|4 +Brand#11|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#11|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#11|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|19|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|45|-|4 +Brand#11|-|PROMO ANODIZED BRASS|-|49|-|4 +Brand#11|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#11|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#11|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#11|-|PROMO BRUSHED STEEL|-|3|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|36|-|4 +Brand#11|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#11|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#11|-|PROMO PLATED BRASS|-|9|-|4 +Brand#11|-|PROMO PLATED BRASS|-|45|-|4 +Brand#11|-|PROMO PLATED NICKEL|-|19|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#11|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#11|-|PROMO POLISHED COPPER|-|14|-|4 +Brand#11|-|PROMO POLISHED COPPER|-|45|-|4 +Brand#11|-|PROMO POLISHED TIN|-|49|-|4 +Brand#11|-|SMALL ANODIZED COPPER|-|36|-|4 +Brand#11|-|SMALL ANODIZED NICKEL|-|3|-|4 +Brand#11|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#11|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|14|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#11|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#11|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#11|-|SMALL BURNISHED COPPER|-|23|-|4 +Brand#11|-|SMALL PLATED COPPER|-|45|-|4 +Brand#11|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#11|-|SMALL PLATED STEEL|-|36|-|4 +Brand#11|-|SMALL PLATED TIN|-|19|-|4 +Brand#11|-|SMALL POLISHED BRASS|-|14|-|4 +Brand#11|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#11|-|SMALL POLISHED COPPER|-|14|-|4 +Brand#11|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#11|-|SMALL POLISHED STEEL|-|9|-|4 +Brand#11|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#11|-|STANDARD BRUSHED NICKEL|-|14|-|4 +Brand#11|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#11|-|STANDARD BURNISHED BRASS|-|3|-|4 +Brand#11|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#11|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#11|-|STANDARD PLATED TIN|-|19|-|4 +Brand#11|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#11|-|STANDARD POLISHED TIN|-|14|-|4 +Brand#11|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#12|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#12|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#12|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#12|-|ECONOMY ANODIZED NICKEL|-|45|-|4 +Brand#12|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#12|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#12|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#12|-|ECONOMY BRUSHED STEEL|-|49|-|4 +Brand#12|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#12|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#12|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#12|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#12|-|ECONOMY POLISHED BRASS|-|14|-|4 +Brand#12|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#12|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#12|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#12|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#12|-|LARGE ANODIZED STEEL|-|49|-|4 +Brand#12|-|LARGE ANODIZED TIN|-|36|-|4 +Brand#12|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#12|-|LARGE BURNISHED BRASS|-|14|-|4 +Brand#12|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#12|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#12|-|LARGE BURNISHED NICKEL|-|45|-|4 +Brand#12|-|LARGE BURNISHED TIN|-|36|-|4 +Brand#12|-|LARGE PLATED BRASS|-|3|-|4 +Brand#12|-|LARGE PLATED STEEL|-|36|-|4 +Brand#12|-|LARGE PLATED STEEL|-|45|-|4 +Brand#12|-|LARGE PLATED TIN|-|23|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#12|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#12|-|LARGE POLISHED STEEL|-|3|-|4 +Brand#12|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#12|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#12|-|MEDIUM ANODIZED NICKEL|-|45|-|4 +Brand#12|-|MEDIUM BRUSHED BRASS|-|19|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#12|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|3|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#12|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#12|-|MEDIUM BURNISHED BRASS|-|3|-|4 +Brand#12|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#12|-|MEDIUM BURNISHED NICKEL|-|19|-|4 +Brand#12|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#12|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|23|-|4 +Brand#12|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#12|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#12|-|PROMO ANODIZED BRASS|-|45|-|4 +Brand#12|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#12|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#12|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#12|-|PROMO ANODIZED TIN|-|19|-|4 +Brand#12|-|PROMO BRUSHED COPPER|-|14|-|4 +Brand#12|-|PROMO BRUSHED COPPER|-|19|-|4 +Brand#12|-|PROMO BRUSHED NICKEL|-|23|-|4 +Brand#12|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#12|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#12|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#12|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#12|-|PROMO BURNISHED TIN|-|14|-|4 +Brand#12|-|PROMO PLATED BRASS|-|36|-|4 +Brand#12|-|PROMO POLISHED COPPER|-|23|-|4 +Brand#12|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#12|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#12|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#12|-|PROMO POLISHED TIN|-|23|-|4 +Brand#12|-|PROMO POLISHED TIN|-|36|-|4 +Brand#12|-|SMALL ANODIZED BRASS|-|36|-|4 +Brand#12|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#12|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#12|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#12|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#12|-|SMALL BRUSHED COPPER|-|36|-|4 +Brand#12|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#12|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#12|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#12|-|SMALL BURNISHED COPPER|-|36|-|4 +Brand#12|-|SMALL PLATED BRASS|-|9|-|4 +Brand#12|-|SMALL POLISHED BRASS|-|49|-|4 +Brand#12|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#12|-|SMALL POLISHED TIN|-|3|-|4 +Brand#12|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#12|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#12|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#12|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#12|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#12|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#12|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#12|-|STANDARD BURNISHED COPPER|-|14|-|4 +Brand#12|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#12|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#12|-|STANDARD BURNISHED TIN|-|3|-|4 +Brand#12|-|STANDARD BURNISHED TIN|-|14|-|4 +Brand#12|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#12|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#12|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#12|-|STANDARD PLATED STEEL|-|45|-|4 +Brand#12|-|STANDARD PLATED TIN|-|9|-|4 +Brand#12|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#12|-|STANDARD POLISHED COPPER|-|3|-|4 +Brand#12|-|STANDARD POLISHED NICKEL|-|23|-|4 +Brand#12|-|STANDARD POLISHED TIN|-|14|-|4 +Brand#13|-|ECONOMY ANODIZED NICKEL|-|14|-|4 +Brand#13|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#13|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#13|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#13|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#13|-|ECONOMY BURNISHED STEEL|-|14|-|4 +Brand#13|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#13|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#13|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#13|-|ECONOMY PLATED NICKEL|-|3|-|4 +Brand#13|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#13|-|ECONOMY PLATED TIN|-|3|-|4 +Brand#13|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#13|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#13|-|ECONOMY POLISHED COPPER|-|49|-|4 +Brand#13|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#13|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#13|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#13|-|LARGE ANODIZED COPPER|-|19|-|4 +Brand#13|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#13|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#13|-|LARGE ANODIZED TIN|-|19|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|3|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|9|-|4 +Brand#13|-|LARGE BRUSHED BRASS|-|19|-|4 +Brand#13|-|LARGE BRUSHED COPPER|-|9|-|4 +Brand#13|-|LARGE BRUSHED COPPER|-|36|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#13|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#13|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#13|-|LARGE BRUSHED TIN|-|49|-|4 +Brand#13|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#13|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#13|-|LARGE PLATED COPPER|-|23|-|4 +Brand#13|-|LARGE PLATED STEEL|-|14|-|4 +Brand#13|-|LARGE PLATED STEEL|-|19|-|4 +Brand#13|-|LARGE PLATED STEEL|-|36|-|4 +Brand#13|-|LARGE PLATED TIN|-|14|-|4 +Brand#13|-|LARGE PLATED TIN|-|45|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|3|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|23|-|4 +Brand#13|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#13|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#13|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#13|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#13|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|19|-|4 +Brand#13|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#13|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#13|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#13|-|MEDIUM BRUSHED NICKEL|-|45|-|4 +Brand#13|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#13|-|MEDIUM BURNISHED STEEL|-|19|-|4 +Brand#13|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#13|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#13|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#13|-|MEDIUM PLATED COPPER|-|23|-|4 +Brand#13|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#13|-|PROMO ANODIZED BRASS|-|14|-|4 +Brand#13|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#13|-|PROMO ANODIZED COPPER|-|45|-|4 +Brand#13|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#13|-|PROMO BRUSHED COPPER|-|49|-|4 +Brand#13|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#13|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#13|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#13|-|PROMO BURNISHED STEEL|-|45|-|4 +Brand#13|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#13|-|PROMO PLATED BRASS|-|14|-|4 +Brand#13|-|PROMO PLATED BRASS|-|19|-|4 +Brand#13|-|PROMO PLATED COPPER|-|3|-|4 +Brand#13|-|PROMO PLATED COPPER|-|19|-|4 +Brand#13|-|PROMO PLATED TIN|-|19|-|4 +Brand#13|-|PROMO POLISHED BRASS|-|49|-|4 +Brand#13|-|PROMO POLISHED STEEL|-|45|-|4 +Brand#13|-|PROMO POLISHED TIN|-|14|-|4 +Brand#13|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#13|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#13|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#13|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#13|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#13|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#13|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#13|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#13|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#13|-|SMALL PLATED BRASS|-|9|-|4 +Brand#13|-|SMALL PLATED TIN|-|45|-|4 +Brand#13|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#13|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#13|-|STANDARD ANODIZED COPPER|-|45|-|4 +Brand#13|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#13|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#13|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#13|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#13|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#13|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#13|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#13|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#13|-|STANDARD BURNISHED BRASS|-|14|-|4 +Brand#13|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#13|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#13|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#13|-|STANDARD PLATED NICKEL|-|23|-|4 +Brand#13|-|STANDARD PLATED TIN|-|9|-|4 +Brand#13|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#13|-|STANDARD POLISHED COPPER|-|9|-|4 +Brand#13|-|STANDARD POLISHED COPPER|-|49|-|4 +Brand#13|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#13|-|STANDARD POLISHED NICKEL|-|19|-|4 +Brand#13|-|STANDARD POLISHED STEEL|-|23|-|4 +Brand#14|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#14|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#14|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#14|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#14|-|ECONOMY BRUSHED BRASS|-|19|-|4 +Brand#14|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#14|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#14|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#14|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#14|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#14|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#14|-|ECONOMY BURNISHED TIN|-|3|-|4 +Brand#14|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#14|-|ECONOMY PLATED COPPER|-|49|-|4 +Brand#14|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#14|-|ECONOMY PLATED TIN|-|9|-|4 +Brand#14|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#14|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#14|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#14|-|LARGE ANODIZED COPPER|-|23|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#14|-|LARGE ANODIZED NICKEL|-|19|-|4 +Brand#14|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#14|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#14|-|LARGE BRUSHED NICKEL|-|45|-|4 +Brand#14|-|LARGE PLATED BRASS|-|3|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|3|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#14|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#14|-|LARGE PLATED TIN|-|49|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|14|-|4 +Brand#14|-|LARGE POLISHED BRASS|-|36|-|4 +Brand#14|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#14|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#14|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#14|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#14|-|MEDIUM ANODIZED NICKEL|-|49|-|4 +Brand#14|-|MEDIUM ANODIZED STEEL|-|23|-|4 +Brand#14|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#14|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#14|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#14|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#14|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#14|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#14|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#14|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#14|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#14|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#14|-|PROMO ANODIZED BRASS|-|49|-|4 +Brand#14|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#14|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#14|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#14|-|PROMO BURNISHED STEEL|-|36|-|4 +Brand#14|-|PROMO PLATED BRASS|-|9|-|4 +Brand#14|-|PROMO PLATED BRASS|-|45|-|4 +Brand#14|-|PROMO PLATED COPPER|-|45|-|4 +Brand#14|-|PROMO PLATED STEEL|-|3|-|4 +Brand#14|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#14|-|PROMO POLISHED COPPER|-|49|-|4 +Brand#14|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#14|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#14|-|SMALL ANODIZED TIN|-|23|-|4 +Brand#14|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#14|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#14|-|SMALL BRUSHED COPPER|-|9|-|4 +Brand#14|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#14|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#14|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#14|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#14|-|SMALL BURNISHED STEEL|-|36|-|4 +Brand#14|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#14|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#14|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#14|-|SMALL PLATED STEEL|-|14|-|4 +Brand#14|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#14|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#14|-|SMALL POLISHED NICKEL|-|9|-|4 +Brand#14|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#14|-|SMALL POLISHED TIN|-|14|-|4 +Brand#14|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#14|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#14|-|STANDARD ANODIZED STEEL|-|9|-|4 +Brand#14|-|STANDARD BRUSHED COPPER|-|45|-|4 +Brand#14|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#14|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#14|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#14|-|STANDARD BURNISHED NICKEL|-|9|-|4 +Brand#14|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#14|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#14|-|STANDARD POLISHED NICKEL|-|3|-|4 +Brand#14|-|STANDARD POLISHED NICKEL|-|9|-|4 +Brand#14|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#15|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#15|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#15|-|ECONOMY ANODIZED STEEL|-|36|-|4 +Brand#15|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#15|-|ECONOMY BRUSHED COPPER|-|14|-|4 +Brand#15|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#15|-|ECONOMY BRUSHED STEEL|-|3|-|4 +Brand#15|-|ECONOMY BRUSHED TIN|-|3|-|4 +Brand#15|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#15|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#15|-|ECONOMY BURNISHED COPPER|-|23|-|4 +Brand#15|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|19|-|4 +Brand#15|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#15|-|LARGE ANODIZED BRASS|-|45|-|4 +Brand#15|-|LARGE ANODIZED COPPER|-|3|-|4 +Brand#15|-|LARGE ANODIZED NICKEL|-|9|-|4 +Brand#15|-|LARGE ANODIZED TIN|-|19|-|4 +Brand#15|-|LARGE BRUSHED BRASS|-|9|-|4 +Brand#15|-|LARGE BRUSHED BRASS|-|19|-|4 +Brand#15|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|14|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#15|-|LARGE BRUSHED STEEL|-|36|-|4 +Brand#15|-|LARGE BURNISHED BRASS|-|14|-|4 +Brand#15|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#15|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#15|-|LARGE BURNISHED COPPER|-|45|-|4 +Brand#15|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#15|-|LARGE PLATED BRASS|-|19|-|4 +Brand#15|-|LARGE PLATED COPPER|-|3|-|4 +Brand#15|-|LARGE PLATED COPPER|-|23|-|4 +Brand#15|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#15|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#15|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#15|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#15|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#15|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#15|-|MEDIUM BRUSHED TIN|-|23|-|4 +Brand#15|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#15|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#15|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#15|-|MEDIUM PLATED NICKEL|-|9|-|4 +Brand#15|-|MEDIUM PLATED NICKEL|-|19|-|4 +Brand#15|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#15|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#15|-|MEDIUM PLATED TIN|-|23|-|4 +Brand#15|-|PROMO ANODIZED COPPER|-|23|-|4 +Brand#15|-|PROMO ANODIZED STEEL|-|14|-|4 +Brand#15|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#15|-|PROMO BRUSHED COPPER|-|14|-|4 +Brand#15|-|PROMO BRUSHED COPPER|-|19|-|4 +Brand#15|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#15|-|PROMO BRUSHED NICKEL|-|23|-|4 +Brand#15|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#15|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#15|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#15|-|PROMO BURNISHED STEEL|-|45|-|4 +Brand#15|-|PROMO PLATED COPPER|-|3|-|4 +Brand#15|-|PROMO PLATED COPPER|-|36|-|4 +Brand#15|-|PROMO PLATED STEEL|-|3|-|4 +Brand#15|-|PROMO PLATED TIN|-|49|-|4 +Brand#15|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#15|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#15|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#15|-|PROMO POLISHED TIN|-|49|-|4 +Brand#15|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#15|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#15|-|SMALL ANODIZED COPPER|-|9|-|4 +Brand#15|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#15|-|SMALL BRUSHED BRASS|-|3|-|4 +Brand#15|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#15|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#15|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#15|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#15|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#15|-|SMALL BURNISHED NICKEL|-|19|-|4 +Brand#15|-|SMALL BURNISHED NICKEL|-|49|-|4 +Brand#15|-|SMALL BURNISHED STEEL|-|9|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|19|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#15|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#15|-|SMALL PLATED BRASS|-|3|-|4 +Brand#15|-|SMALL PLATED COPPER|-|23|-|4 +Brand#15|-|SMALL PLATED COPPER|-|49|-|4 +Brand#15|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#15|-|SMALL PLATED NICKEL|-|45|-|4 +Brand#15|-|SMALL PLATED STEEL|-|3|-|4 +Brand#15|-|SMALL PLATED TIN|-|9|-|4 +Brand#15|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#15|-|SMALL POLISHED NICKEL|-|3|-|4 +Brand#15|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#15|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#15|-|SMALL POLISHED TIN|-|19|-|4 +Brand#15|-|SMALL POLISHED TIN|-|49|-|4 +Brand#15|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#15|-|STANDARD ANODIZED NICKEL|-|49|-|4 +Brand#15|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#15|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#15|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#15|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#15|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#15|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#15|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#15|-|STANDARD PLATED COPPER|-|23|-|4 +Brand#15|-|STANDARD PLATED NICKEL|-|19|-|4 +Brand#15|-|STANDARD PLATED TIN|-|45|-|4 +Brand#15|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#15|-|STANDARD POLISHED COPPER|-|23|-|4 +Brand#15|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#21|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#21|-|ECONOMY ANODIZED NICKEL|-|14|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#21|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#21|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|9|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#21|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#21|-|ECONOMY BRUSHED COPPER|-|49|-|4 +Brand#21|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#21|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#21|-|ECONOMY BURNISHED BRASS|-|3|-|4 +Brand#21|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#21|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#21|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#21|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#21|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#21|-|ECONOMY PLATED COPPER|-|14|-|4 +Brand#21|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#21|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#21|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#21|-|LARGE ANODIZED COPPER|-|3|-|4 +Brand#21|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#21|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#21|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#21|-|LARGE BRUSHED COPPER|-|45|-|4 +Brand#21|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#21|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#21|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#21|-|LARGE BURNISHED STEEL|-|49|-|4 +Brand#21|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#21|-|LARGE PLATED BRASS|-|19|-|4 +Brand#21|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#21|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#21|-|LARGE PLATED TIN|-|19|-|4 +Brand#21|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#21|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#21|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#21|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#21|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#21|-|LARGE POLISHED TIN|-|49|-|4 +Brand#21|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#21|-|MEDIUM ANODIZED NICKEL|-|49|-|4 +Brand#21|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#21|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#21|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#21|-|MEDIUM BRUSHED TIN|-|3|-|4 +Brand#21|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#21|-|MEDIUM BURNISHED NICKEL|-|14|-|4 +Brand#21|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#21|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#21|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#21|-|MEDIUM PLATED BRASS|-|19|-|4 +Brand#21|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#21|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#21|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#21|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|3|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|14|-|4 +Brand#21|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#21|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#21|-|PROMO BRUSHED STEEL|-|45|-|4 +Brand#21|-|PROMO BURNISHED BRASS|-|19|-|4 +Brand#21|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#21|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#21|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#21|-|PROMO PLATED NICKEL|-|9|-|4 +Brand#21|-|PROMO PLATED NICKEL|-|36|-|4 +Brand#21|-|PROMO PLATED STEEL|-|49|-|4 +Brand#21|-|PROMO PLATED TIN|-|3|-|4 +Brand#21|-|PROMO POLISHED NICKEL|-|23|-|4 +Brand#21|-|PROMO POLISHED TIN|-|14|-|4 +Brand#21|-|PROMO POLISHED TIN|-|19|-|4 +Brand#21|-|PROMO POLISHED TIN|-|23|-|4 +Brand#21|-|SMALL BRUSHED BRASS|-|23|-|4 +Brand#21|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#21|-|SMALL BURNISHED BRASS|-|23|-|4 +Brand#21|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#21|-|SMALL BURNISHED STEEL|-|19|-|4 +Brand#21|-|SMALL BURNISHED TIN|-|19|-|4 +Brand#21|-|SMALL PLATED BRASS|-|45|-|4 +Brand#21|-|SMALL PLATED COPPER|-|45|-|4 +Brand#21|-|SMALL PLATED STEEL|-|45|-|4 +Brand#21|-|SMALL PLATED TIN|-|14|-|4 +Brand#21|-|SMALL PLATED TIN|-|45|-|4 +Brand#21|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#21|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#21|-|SMALL POLISHED TIN|-|3|-|4 +Brand#21|-|STANDARD ANODIZED BRASS|-|9|-|4 +Brand#21|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#21|-|STANDARD ANODIZED TIN|-|45|-|4 +Brand#21|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#21|-|STANDARD BURNISHED NICKEL|-|23|-|4 +Brand#21|-|STANDARD BURNISHED TIN|-|9|-|4 +Brand#21|-|STANDARD PLATED BRASS|-|14|-|4 +Brand#21|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#21|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#21|-|STANDARD PLATED STEEL|-|9|-|4 +Brand#21|-|STANDARD PLATED TIN|-|9|-|4 +Brand#21|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#21|-|STANDARD POLISHED COPPER|-|49|-|4 +Brand#21|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#21|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|14|-|4 +Brand#22|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#22|-|ECONOMY ANODIZED TIN|-|9|-|4 +Brand#22|-|ECONOMY ANODIZED TIN|-|36|-|4 +Brand#22|-|ECONOMY BRUSHED NICKEL|-|36|-|4 +Brand#22|-|ECONOMY BRUSHED NICKEL|-|45|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|9|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|23|-|4 +Brand#22|-|ECONOMY BURNISHED BRASS|-|45|-|4 +Brand#22|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#22|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|9|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|14|-|4 +Brand#22|-|ECONOMY BURNISHED STEEL|-|23|-|4 +Brand#22|-|ECONOMY PLATED BRASS|-|36|-|4 +Brand#22|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#22|-|ECONOMY PLATED TIN|-|3|-|4 +Brand#22|-|ECONOMY POLISHED TIN|-|49|-|4 +Brand#22|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#22|-|LARGE ANODIZED COPPER|-|36|-|4 +Brand#22|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#22|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#22|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#22|-|LARGE BRUSHED STEEL|-|49|-|4 +Brand#22|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#22|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#22|-|LARGE BURNISHED STEEL|-|45|-|4 +Brand#22|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#22|-|LARGE PLATED COPPER|-|14|-|4 +Brand#22|-|LARGE PLATED STEEL|-|49|-|4 +Brand#22|-|LARGE POLISHED BRASS|-|19|-|4 +Brand#22|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#22|-|LARGE POLISHED COPPER|-|23|-|4 +Brand#22|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#22|-|LARGE POLISHED TIN|-|49|-|4 +Brand#22|-|MEDIUM ANODIZED BRASS|-|45|-|4 +Brand#22|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#22|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#22|-|MEDIUM ANODIZED NICKEL|-|36|-|4 +Brand#22|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#22|-|MEDIUM ANODIZED TIN|-|9|-|4 +Brand#22|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#22|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#22|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#22|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#22|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#22|-|MEDIUM BRUSHED TIN|-|45|-|4 +Brand#22|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|19|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#22|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#22|-|MEDIUM PLATED BRASS|-|49|-|4 +Brand#22|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#22|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#22|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#22|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#22|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|9|-|4 +Brand#22|-|PROMO BRUSHED BRASS|-|36|-|4 +Brand#22|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#22|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#22|-|PROMO BURNISHED COPPER|-|9|-|4 +Brand#22|-|PROMO PLATED BRASS|-|14|-|4 +Brand#22|-|PROMO PLATED BRASS|-|45|-|4 +Brand#22|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#22|-|PROMO PLATED STEEL|-|19|-|4 +Brand#22|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#22|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#22|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#22|-|SMALL ANODIZED TIN|-|36|-|4 +Brand#22|-|SMALL ANODIZED TIN|-|49|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|3|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|36|-|4 +Brand#22|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#22|-|SMALL BRUSHED TIN|-|45|-|4 +Brand#22|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#22|-|SMALL BURNISHED TIN|-|14|-|4 +Brand#22|-|SMALL PLATED STEEL|-|3|-|4 +Brand#22|-|SMALL PLATED TIN|-|9|-|4 +Brand#22|-|SMALL PLATED TIN|-|36|-|4 +Brand#22|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#22|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#22|-|STANDARD ANODIZED BRASS|-|14|-|4 +Brand#22|-|STANDARD ANODIZED BRASS|-|23|-|4 +Brand#22|-|STANDARD BRUSHED COPPER|-|49|-|4 +Brand#22|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#22|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#22|-|STANDARD BRUSHED STEEL|-|9|-|4 +Brand#22|-|STANDARD BRUSHED TIN|-|19|-|4 +Brand#22|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|3|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|14|-|4 +Brand#22|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#22|-|STANDARD BURNISHED STEEL|-|49|-|4 +Brand#22|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#22|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#22|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#22|-|STANDARD PLATED STEEL|-|14|-|4 +Brand#22|-|STANDARD PLATED TIN|-|19|-|4 +Brand#22|-|STANDARD PLATED TIN|-|49|-|4 +Brand#22|-|STANDARD POLISHED COPPER|-|9|-|4 +Brand#22|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#22|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#23|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#23|-|ECONOMY ANODIZED STEEL|-|14|-|4 +Brand#23|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#23|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#23|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#23|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#23|-|ECONOMY BRUSHED TIN|-|9|-|4 +Brand#23|-|ECONOMY BURNISHED STEEL|-|49|-|4 +Brand#23|-|ECONOMY PLATED COPPER|-|14|-|4 +Brand#23|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#23|-|ECONOMY PLATED STEEL|-|14|-|4 +Brand#23|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#23|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#23|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|19|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#23|-|LARGE ANODIZED STEEL|-|49|-|4 +Brand#23|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#23|-|LARGE PLATED BRASS|-|9|-|4 +Brand#23|-|LARGE PLATED BRASS|-|49|-|4 +Brand#23|-|LARGE PLATED COPPER|-|3|-|4 +Brand#23|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#23|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#23|-|MEDIUM ANODIZED BRASS|-|19|-|4 +Brand#23|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#23|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#23|-|MEDIUM ANODIZED STEEL|-|45|-|4 +Brand#23|-|MEDIUM ANODIZED TIN|-|36|-|4 +Brand#23|-|MEDIUM ANODIZED TIN|-|45|-|4 +Brand#23|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#23|-|MEDIUM BRUSHED COPPER|-|23|-|4 +Brand#23|-|MEDIUM BRUSHED NICKEL|-|3|-|4 +Brand#23|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#23|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#23|-|MEDIUM BURNISHED BRASS|-|45|-|4 +Brand#23|-|MEDIUM BURNISHED COPPER|-|19|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|36|-|4 +Brand#23|-|MEDIUM PLATED COPPER|-|45|-|4 +Brand#23|-|MEDIUM PLATED NICKEL|-|9|-|4 +Brand#23|-|MEDIUM PLATED NICKEL|-|14|-|4 +Brand#23|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#23|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#23|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#23|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|19|-|4 +Brand#23|-|PROMO BRUSHED BRASS|-|36|-|4 +Brand#23|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#23|-|PROMO BRUSHED TIN|-|49|-|4 +Brand#23|-|PROMO BURNISHED BRASS|-|14|-|4 +Brand#23|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#23|-|PROMO BURNISHED COPPER|-|14|-|4 +Brand#23|-|PROMO PLATED BRASS|-|23|-|4 +Brand#23|-|PROMO POLISHED BRASS|-|14|-|4 +Brand#23|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#23|-|PROMO POLISHED COPPER|-|36|-|4 +Brand#23|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#23|-|SMALL ANODIZED BRASS|-|23|-|4 +Brand#23|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#23|-|SMALL BRUSHED BRASS|-|49|-|4 +Brand#23|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#23|-|SMALL BRUSHED STEEL|-|3|-|4 +Brand#23|-|SMALL BRUSHED STEEL|-|19|-|4 +Brand#23|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#23|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#23|-|SMALL BURNISHED COPPER|-|49|-|4 +Brand#23|-|SMALL BURNISHED STEEL|-|45|-|4 +Brand#23|-|SMALL PLATED BRASS|-|36|-|4 +Brand#23|-|SMALL PLATED BRASS|-|49|-|4 +Brand#23|-|SMALL PLATED COPPER|-|14|-|4 +Brand#23|-|SMALL PLATED TIN|-|14|-|4 +Brand#23|-|SMALL POLISHED BRASS|-|9|-|4 +Brand#23|-|SMALL POLISHED BRASS|-|14|-|4 +Brand#23|-|SMALL POLISHED NICKEL|-|3|-|4 +Brand#23|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#23|-|SMALL POLISHED TIN|-|9|-|4 +Brand#23|-|STANDARD ANODIZED BRASS|-|19|-|4 +Brand#23|-|STANDARD ANODIZED BRASS|-|45|-|4 +Brand#23|-|STANDARD ANODIZED COPPER|-|19|-|4 +Brand#23|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#23|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#23|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#23|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#23|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#23|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#23|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#23|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#23|-|STANDARD PLATED TIN|-|19|-|4 +Brand#23|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#23|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#23|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#23|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#23|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#24|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#24|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#24|-|ECONOMY ANODIZED COPPER|-|49|-|4 +Brand#24|-|ECONOMY BRUSHED BRASS|-|36|-|4 +Brand#24|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#24|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#24|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#24|-|ECONOMY BURNISHED STEEL|-|45|-|4 +Brand#24|-|ECONOMY PLATED BRASS|-|23|-|4 +Brand#24|-|ECONOMY PLATED COPPER|-|36|-|4 +Brand#24|-|ECONOMY PLATED STEEL|-|45|-|4 +Brand#24|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#24|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#24|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#24|-|ECONOMY POLISHED STEEL|-|14|-|4 +Brand#24|-|ECONOMY POLISHED STEEL|-|36|-|4 +Brand#24|-|LARGE ANODIZED NICKEL|-|23|-|4 +Brand#24|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#24|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#24|-|LARGE BRUSHED BRASS|-|14|-|4 +Brand#24|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#24|-|LARGE BRUSHED STEEL|-|45|-|4 +Brand#24|-|LARGE BRUSHED TIN|-|49|-|4 +Brand#24|-|LARGE BURNISHED BRASS|-|3|-|4 +Brand#24|-|LARGE BURNISHED NICKEL|-|19|-|4 +Brand#24|-|LARGE PLATED BRASS|-|9|-|4 +Brand#24|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#24|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#24|-|LARGE PLATED TIN|-|9|-|4 +Brand#24|-|LARGE PLATED TIN|-|19|-|4 +Brand#24|-|LARGE PLATED TIN|-|36|-|4 +Brand#24|-|LARGE PLATED TIN|-|49|-|4 +Brand#24|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#24|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#24|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#24|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#24|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#24|-|LARGE POLISHED TIN|-|14|-|4 +Brand#24|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#24|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#24|-|MEDIUM BRUSHED COPPER|-|14|-|4 +Brand#24|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#24|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|45|-|4 +Brand#24|-|MEDIUM BRUSHED STEEL|-|49|-|4 +Brand#24|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#24|-|MEDIUM BURNISHED NICKEL|-|36|-|4 +Brand#24|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#24|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#24|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#24|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#24|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#24|-|PROMO ANODIZED STEEL|-|3|-|4 +Brand#24|-|PROMO ANODIZED TIN|-|45|-|4 +Brand#24|-|PROMO BRUSHED BRASS|-|19|-|4 +Brand#24|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#24|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#24|-|PROMO BRUSHED STEEL|-|49|-|4 +Brand#24|-|PROMO BURNISHED BRASS|-|3|-|4 +Brand#24|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#24|-|PROMO BURNISHED STEEL|-|49|-|4 +Brand#24|-|PROMO PLATED BRASS|-|3|-|4 +Brand#24|-|PROMO PLATED COPPER|-|23|-|4 +Brand#24|-|PROMO PLATED COPPER|-|49|-|4 +Brand#24|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#24|-|PROMO POLISHED BRASS|-|14|-|4 +Brand#24|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#24|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#24|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#24|-|SMALL ANODIZED COPPER|-|3|-|4 +Brand#24|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#24|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#24|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#24|-|SMALL ANODIZED TIN|-|36|-|4 +Brand#24|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#24|-|SMALL BRUSHED NICKEL|-|49|-|4 +Brand#24|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#24|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#24|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#24|-|SMALL PLATED BRASS|-|3|-|4 +Brand#24|-|SMALL PLATED COPPER|-|14|-|4 +Brand#24|-|SMALL PLATED COPPER|-|36|-|4 +Brand#24|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#24|-|SMALL PLATED NICKEL|-|49|-|4 +Brand#24|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|9|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#24|-|SMALL POLISHED NICKEL|-|36|-|4 +Brand#24|-|SMALL POLISHED STEEL|-|9|-|4 +Brand#24|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#24|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#24|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#24|-|STANDARD BRUSHED BRASS|-|14|-|4 +Brand#24|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#24|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#24|-|STANDARD BRUSHED STEEL|-|14|-|4 +Brand#24|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#24|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#24|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#24|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#24|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#24|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#24|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#24|-|STANDARD PLATED TIN|-|36|-|4 +Brand#24|-|STANDARD POLISHED COPPER|-|45|-|4 +Brand#24|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#25|-|ECONOMY ANODIZED BRASS|-|14|-|4 +Brand#25|-|ECONOMY ANODIZED BRASS|-|49|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|9|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#25|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#25|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#25|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#25|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#25|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#25|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#25|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#25|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#25|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#25|-|ECONOMY POLISHED TIN|-|9|-|4 +Brand#25|-|ECONOMY POLISHED TIN|-|45|-|4 +Brand#25|-|LARGE ANODIZED BRASS|-|3|-|4 +Brand#25|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#25|-|LARGE ANODIZED COPPER|-|36|-|4 +Brand#25|-|LARGE ANODIZED NICKEL|-|23|-|4 +Brand#25|-|LARGE ANODIZED STEEL|-|23|-|4 +Brand#25|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#25|-|LARGE BRUSHED NICKEL|-|49|-|4 +Brand#25|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#25|-|LARGE BRUSHED TIN|-|9|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#25|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#25|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#25|-|LARGE BURNISHED TIN|-|49|-|4 +Brand#25|-|LARGE PLATED BRASS|-|14|-|4 +Brand#25|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#25|-|LARGE PLATED NICKEL|-|45|-|4 +Brand#25|-|LARGE PLATED TIN|-|19|-|4 +Brand#25|-|LARGE PLATED TIN|-|23|-|4 +Brand#25|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#25|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#25|-|LARGE POLISHED COPPER|-|36|-|4 +Brand#25|-|MEDIUM ANODIZED TIN|-|36|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#25|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED NICKEL|-|9|-|4 +Brand#25|-|MEDIUM BURNISHED NICKEL|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#25|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#25|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#25|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#25|-|MEDIUM PLATED BRASS|-|45|-|4 +Brand#25|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#25|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#25|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#25|-|MEDIUM PLATED STEEL|-|36|-|4 +Brand#25|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#25|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#25|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#25|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#25|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#25|-|PROMO BRUSHED NICKEL|-|3|-|4 +Brand#25|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#25|-|PROMO BRUSHED TIN|-|14|-|4 +Brand#25|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|19|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|45|-|4 +Brand#25|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#25|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#25|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#25|-|PROMO PLATED BRASS|-|45|-|4 +Brand#25|-|PROMO PLATED COPPER|-|19|-|4 +Brand#25|-|PROMO PLATED NICKEL|-|45|-|4 +Brand#25|-|PROMO PLATED NICKEL|-|49|-|4 +Brand#25|-|PROMO PLATED STEEL|-|23|-|4 +Brand#25|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#25|-|SMALL ANODIZED BRASS|-|45|-|4 +Brand#25|-|SMALL ANODIZED NICKEL|-|19|-|4 +Brand#25|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#25|-|SMALL ANODIZED TIN|-|14|-|4 +Brand#25|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#25|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#25|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#25|-|SMALL BURNISHED COPPER|-|3|-|4 +Brand#25|-|SMALL BURNISHED STEEL|-|3|-|4 +Brand#25|-|SMALL BURNISHED STEEL|-|14|-|4 +Brand#25|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#25|-|SMALL PLATED BRASS|-|19|-|4 +Brand#25|-|SMALL PLATED COPPER|-|23|-|4 +Brand#25|-|SMALL PLATED STEEL|-|45|-|4 +Brand#25|-|SMALL PLATED TIN|-|36|-|4 +Brand#25|-|SMALL POLISHED BRASS|-|23|-|4 +Brand#25|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#25|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#25|-|STANDARD ANODIZED STEEL|-|3|-|4 +Brand#25|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#25|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#25|-|STANDARD BRUSHED BRASS|-|14|-|4 +Brand#25|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#25|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#25|-|STANDARD BURNISHED NICKEL|-|9|-|4 +Brand#25|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#25|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#25|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#25|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#25|-|STANDARD POLISHED COPPER|-|23|-|4 +Brand#25|-|STANDARD POLISHED NICKEL|-|3|-|4 +Brand#25|-|STANDARD POLISHED NICKEL|-|49|-|4 +Brand#25|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#25|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#31|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#31|-|ECONOMY ANODIZED COPPER|-|45|-|4 +Brand#31|-|ECONOMY ANODIZED STEEL|-|3|-|4 +Brand#31|-|ECONOMY ANODIZED TIN|-|45|-|4 +Brand#31|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#31|-|ECONOMY BRUSHED COPPER|-|19|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|9|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#31|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#31|-|ECONOMY BURNISHED COPPER|-|36|-|4 +Brand#31|-|ECONOMY BURNISHED STEEL|-|3|-|4 +Brand#31|-|ECONOMY BURNISHED TIN|-|49|-|4 +Brand#31|-|ECONOMY PLATED COPPER|-|49|-|4 +Brand#31|-|ECONOMY PLATED NICKEL|-|9|-|4 +Brand#31|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#31|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#31|-|ECONOMY PLATED TIN|-|49|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|3|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|36|-|4 +Brand#31|-|ECONOMY POLISHED COPPER|-|49|-|4 +Brand#31|-|ECONOMY POLISHED NICKEL|-|3|-|4 +Brand#31|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#31|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#31|-|LARGE BRUSHED BRASS|-|36|-|4 +Brand#31|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#31|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#31|-|LARGE BURNISHED BRASS|-|9|-|4 +Brand#31|-|LARGE PLATED COPPER|-|19|-|4 +Brand#31|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#31|-|LARGE PLATED TIN|-|9|-|4 +Brand#31|-|LARGE PLATED TIN|-|14|-|4 +Brand#31|-|LARGE POLISHED BRASS|-|14|-|4 +Brand#31|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#31|-|LARGE POLISHED STEEL|-|45|-|4 +Brand#31|-|LARGE POLISHED TIN|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#31|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#31|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#31|-|MEDIUM ANODIZED STEEL|-|49|-|4 +Brand#31|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#31|-|MEDIUM ANODIZED TIN|-|49|-|4 +Brand#31|-|MEDIUM BRUSHED BRASS|-|36|-|4 +Brand#31|-|MEDIUM BRUSHED STEEL|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED BRASS|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED COPPER|-|3|-|4 +Brand#31|-|MEDIUM BURNISHED NICKEL|-|9|-|4 +Brand#31|-|MEDIUM BURNISHED STEEL|-|9|-|4 +Brand#31|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#31|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#31|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|36|-|4 +Brand#31|-|MEDIUM PLATED TIN|-|45|-|4 +Brand#31|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#31|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#31|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#31|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#31|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#31|-|PROMO BRUSHED NICKEL|-|45|-|4 +Brand#31|-|PROMO BURNISHED COPPER|-|36|-|4 +Brand#31|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#31|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#31|-|PROMO PLATED BRASS|-|19|-|4 +Brand#31|-|PROMO PLATED NICKEL|-|36|-|4 +Brand#31|-|PROMO POLISHED BRASS|-|49|-|4 +Brand#31|-|PROMO POLISHED COPPER|-|14|-|4 +Brand#31|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#31|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#31|-|PROMO POLISHED TIN|-|3|-|4 +Brand#31|-|PROMO POLISHED TIN|-|23|-|4 +Brand#31|-|SMALL ANODIZED COPPER|-|45|-|4 +Brand#31|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#31|-|SMALL ANODIZED TIN|-|3|-|4 +Brand#31|-|SMALL BRUSHED COPPER|-|36|-|4 +Brand#31|-|SMALL BRUSHED COPPER|-|49|-|4 +Brand#31|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#31|-|SMALL BRUSHED NICKEL|-|23|-|4 +Brand#31|-|SMALL BURNISHED BRASS|-|45|-|4 +Brand#31|-|SMALL BURNISHED NICKEL|-|9|-|4 +Brand#31|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#31|-|SMALL PLATED COPPER|-|36|-|4 +Brand#31|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#31|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#31|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#31|-|SMALL POLISHED COPPER|-|45|-|4 +Brand#31|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#31|-|SMALL POLISHED TIN|-|23|-|4 +Brand#31|-|SMALL POLISHED TIN|-|49|-|4 +Brand#31|-|STANDARD BRUSHED STEEL|-|23|-|4 +Brand#31|-|STANDARD BRUSHED STEEL|-|49|-|4 +Brand#31|-|STANDARD BURNISHED BRASS|-|14|-|4 +Brand#31|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#31|-|STANDARD PLATED NICKEL|-|3|-|4 +Brand#31|-|STANDARD POLISHED BRASS|-|3|-|4 +Brand#31|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#31|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED BRASS|-|19|-|4 +Brand#32|-|ECONOMY ANODIZED COPPER|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|36|-|4 +Brand#32|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#32|-|ECONOMY ANODIZED TIN|-|19|-|4 +Brand#32|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#32|-|ECONOMY BRUSHED TIN|-|45|-|4 +Brand#32|-|ECONOMY BURNISHED BRASS|-|23|-|4 +Brand#32|-|ECONOMY BURNISHED COPPER|-|36|-|4 +Brand#32|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#32|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#32|-|ECONOMY PLATED BRASS|-|9|-|4 +Brand#32|-|ECONOMY PLATED COPPER|-|9|-|4 +Brand#32|-|ECONOMY PLATED NICKEL|-|23|-|4 +Brand#32|-|ECONOMY PLATED TIN|-|45|-|4 +Brand#32|-|ECONOMY POLISHED STEEL|-|3|-|4 +Brand#32|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#32|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#32|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#32|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#32|-|LARGE ANODIZED STEEL|-|14|-|4 +Brand#32|-|LARGE BRUSHED STEEL|-|45|-|4 +Brand#32|-|LARGE BRUSHED TIN|-|45|-|4 +Brand#32|-|LARGE BURNISHED NICKEL|-|36|-|4 +Brand#32|-|LARGE BURNISHED TIN|-|19|-|4 +Brand#32|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#32|-|LARGE PLATED BRASS|-|3|-|4 +Brand#32|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#32|-|LARGE PLATED STEEL|-|19|-|4 +Brand#32|-|LARGE PLATED STEEL|-|36|-|4 +Brand#32|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#32|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#32|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#32|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#32|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#32|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#32|-|MEDIUM ANODIZED TIN|-|23|-|4 +Brand#32|-|MEDIUM BRUSHED BRASS|-|23|-|4 +Brand#32|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#32|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#32|-|MEDIUM BRUSHED COPPER|-|19|-|4 +Brand#32|-|MEDIUM BRUSHED TIN|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#32|-|MEDIUM BURNISHED BRASS|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED COPPER|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#32|-|MEDIUM BURNISHED NICKEL|-|49|-|4 +Brand#32|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#32|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#32|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#32|-|MEDIUM PLATED BRASS|-|49|-|4 +Brand#32|-|MEDIUM PLATED COPPER|-|3|-|4 +Brand#32|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#32|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#32|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#32|-|PROMO ANODIZED COPPER|-|19|-|4 +Brand#32|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#32|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#32|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#32|-|PROMO BRUSHED NICKEL|-|36|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#32|-|PROMO BRUSHED STEEL|-|49|-|4 +Brand#32|-|PROMO BURNISHED BRASS|-|45|-|4 +Brand#32|-|PROMO BURNISHED NICKEL|-|45|-|4 +Brand#32|-|PROMO BURNISHED TIN|-|14|-|4 +Brand#32|-|PROMO BURNISHED TIN|-|45|-|4 +Brand#32|-|PROMO PLATED TIN|-|19|-|4 +Brand#32|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#32|-|PROMO POLISHED TIN|-|3|-|4 +Brand#32|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#32|-|SMALL ANODIZED NICKEL|-|3|-|4 +Brand#32|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#32|-|SMALL ANODIZED TIN|-|9|-|4 +Brand#32|-|SMALL BRUSHED BRASS|-|9|-|4 +Brand#32|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#32|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#32|-|SMALL BRUSHED COPPER|-|23|-|4 +Brand#32|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#32|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#32|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#32|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#32|-|SMALL BURNISHED NICKEL|-|36|-|4 +Brand#32|-|SMALL BURNISHED STEEL|-|3|-|4 +Brand#32|-|SMALL BURNISHED TIN|-|23|-|4 +Brand#32|-|SMALL PLATED BRASS|-|49|-|4 +Brand#32|-|SMALL PLATED COPPER|-|36|-|4 +Brand#32|-|SMALL PLATED COPPER|-|45|-|4 +Brand#32|-|SMALL PLATED NICKEL|-|45|-|4 +Brand#32|-|SMALL PLATED STEEL|-|45|-|4 +Brand#32|-|SMALL PLATED TIN|-|23|-|4 +Brand#32|-|SMALL PLATED TIN|-|36|-|4 +Brand#32|-|SMALL PLATED TIN|-|45|-|4 +Brand#32|-|SMALL POLISHED NICKEL|-|36|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|14|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|23|-|4 +Brand#32|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#32|-|SMALL POLISHED TIN|-|36|-|4 +Brand#32|-|SMALL POLISHED TIN|-|45|-|4 +Brand#32|-|STANDARD ANODIZED NICKEL|-|19|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|14|-|4 +Brand#32|-|STANDARD ANODIZED TIN|-|19|-|4 +Brand#32|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#32|-|STANDARD BURNISHED BRASS|-|36|-|4 +Brand#32|-|STANDARD BURNISHED BRASS|-|45|-|4 +Brand#32|-|STANDARD BURNISHED COPPER|-|3|-|4 +Brand#32|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#32|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#32|-|STANDARD BURNISHED STEEL|-|49|-|4 +Brand#32|-|STANDARD BURNISHED TIN|-|23|-|4 +Brand#32|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#32|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#32|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#32|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#32|-|STANDARD POLISHED COPPER|-|36|-|4 +Brand#32|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#33|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#33|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|9|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|23|-|4 +Brand#33|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#33|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#33|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#33|-|ECONOMY BURNISHED BRASS|-|49|-|4 +Brand#33|-|ECONOMY BURNISHED COPPER|-|3|-|4 +Brand#33|-|ECONOMY BURNISHED COPPER|-|14|-|4 +Brand#33|-|ECONOMY BURNISHED STEEL|-|3|-|4 +Brand#33|-|ECONOMY BURNISHED TIN|-|36|-|4 +Brand#33|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#33|-|ECONOMY PLATED COPPER|-|19|-|4 +Brand#33|-|ECONOMY PLATED COPPER|-|45|-|4 +Brand#33|-|ECONOMY PLATED NICKEL|-|14|-|4 +Brand#33|-|ECONOMY PLATED NICKEL|-|36|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#33|-|ECONOMY PLATED STEEL|-|36|-|4 +Brand#33|-|ECONOMY POLISHED BRASS|-|14|-|4 +Brand#33|-|ECONOMY POLISHED NICKEL|-|19|-|4 +Brand#33|-|ECONOMY POLISHED TIN|-|9|-|4 +Brand#33|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#33|-|LARGE ANODIZED COPPER|-|19|-|4 +Brand#33|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#33|-|LARGE ANODIZED NICKEL|-|36|-|4 +Brand#33|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#33|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#33|-|LARGE ANODIZED STEEL|-|45|-|4 +Brand#33|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#33|-|LARGE BRUSHED BRASS|-|3|-|4 +Brand#33|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#33|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#33|-|LARGE BRUSHED TIN|-|36|-|4 +Brand#33|-|LARGE BURNISHED COPPER|-|45|-|4 +Brand#33|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#33|-|LARGE BURNISHED STEEL|-|19|-|4 +Brand#33|-|LARGE PLATED BRASS|-|3|-|4 +Brand#33|-|LARGE PLATED COPPER|-|19|-|4 +Brand#33|-|LARGE PLATED STEEL|-|3|-|4 +Brand#33|-|LARGE PLATED STEEL|-|19|-|4 +Brand#33|-|LARGE PLATED TIN|-|45|-|4 +Brand#33|-|LARGE POLISHED BRASS|-|45|-|4 +Brand#33|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#33|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#33|-|LARGE POLISHED TIN|-|23|-|4 +Brand#33|-|MEDIUM ANODIZED BRASS|-|3|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#33|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#33|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#33|-|MEDIUM ANODIZED NICKEL|-|19|-|4 +Brand#33|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#33|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#33|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#33|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#33|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#33|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#33|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#33|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#33|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#33|-|PROMO ANODIZED BRASS|-|3|-|4 +Brand#33|-|PROMO BRUSHED BRASS|-|49|-|4 +Brand#33|-|PROMO BURNISHED COPPER|-|23|-|4 +Brand#33|-|PROMO BURNISHED NICKEL|-|14|-|4 +Brand#33|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#33|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#33|-|PROMO BURNISHED TIN|-|23|-|4 +Brand#33|-|PROMO PLATED COPPER|-|14|-|4 +Brand#33|-|PROMO PLATED STEEL|-|45|-|4 +Brand#33|-|PROMO PLATED STEEL|-|49|-|4 +Brand#33|-|PROMO PLATED TIN|-|49|-|4 +Brand#33|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|3|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|9|-|4 +Brand#33|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#33|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#33|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#33|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#33|-|SMALL ANODIZED STEEL|-|9|-|4 +Brand#33|-|SMALL BRUSHED BRASS|-|3|-|4 +Brand#33|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#33|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#33|-|SMALL BRUSHED STEEL|-|3|-|4 +Brand#33|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#33|-|SMALL BURNISHED BRASS|-|19|-|4 +Brand#33|-|SMALL BURNISHED NICKEL|-|3|-|4 +Brand#33|-|SMALL PLATED BRASS|-|3|-|4 +Brand#33|-|SMALL PLATED STEEL|-|14|-|4 +Brand#33|-|SMALL PLATED STEEL|-|45|-|4 +Brand#33|-|SMALL PLATED TIN|-|23|-|4 +Brand#33|-|SMALL PLATED TIN|-|36|-|4 +Brand#33|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#33|-|SMALL POLISHED TIN|-|19|-|4 +Brand#33|-|SMALL POLISHED TIN|-|23|-|4 +Brand#33|-|SMALL POLISHED TIN|-|45|-|4 +Brand#33|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#33|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#33|-|STANDARD ANODIZED TIN|-|45|-|4 +Brand#33|-|STANDARD BRUSHED BRASS|-|9|-|4 +Brand#33|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#33|-|STANDARD BRUSHED STEEL|-|9|-|4 +Brand#33|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#33|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#33|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#33|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#33|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#33|-|STANDARD PLATED COPPER|-|3|-|4 +Brand#33|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#33|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#33|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#33|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#33|-|STANDARD PLATED TIN|-|14|-|4 +Brand#33|-|STANDARD POLISHED BRASS|-|9|-|4 +Brand#33|-|STANDARD POLISHED BRASS|-|19|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#33|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#34|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|3|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|14|-|4 +Brand#34|-|ECONOMY ANODIZED COPPER|-|19|-|4 +Brand#34|-|ECONOMY ANODIZED STEEL|-|9|-|4 +Brand#34|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#34|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#34|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#34|-|ECONOMY BURNISHED COPPER|-|9|-|4 +Brand#34|-|ECONOMY BURNISHED STEEL|-|19|-|4 +Brand#34|-|ECONOMY BURNISHED TIN|-|3|-|4 +Brand#34|-|ECONOMY BURNISHED TIN|-|23|-|4 +Brand#34|-|ECONOMY PLATED BRASS|-|9|-|4 +Brand#34|-|ECONOMY PLATED BRASS|-|14|-|4 +Brand#34|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#34|-|ECONOMY PLATED NICKEL|-|45|-|4 +Brand#34|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#34|-|ECONOMY PLATED TIN|-|45|-|4 +Brand#34|-|ECONOMY POLISHED BRASS|-|45|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#34|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#34|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#34|-|LARGE ANODIZED TIN|-|49|-|4 +Brand#34|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#34|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#34|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#34|-|LARGE BRUSHED STEEL|-|14|-|4 +Brand#34|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#34|-|LARGE BRUSHED TIN|-|9|-|4 +Brand#34|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#34|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#34|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#34|-|LARGE BURNISHED NICKEL|-|19|-|4 +Brand#34|-|LARGE PLATED BRASS|-|23|-|4 +Brand#34|-|LARGE PLATED BRASS|-|36|-|4 +Brand#34|-|LARGE PLATED BRASS|-|45|-|4 +Brand#34|-|LARGE PLATED COPPER|-|23|-|4 +Brand#34|-|LARGE PLATED COPPER|-|49|-|4 +Brand#34|-|LARGE PLATED STEEL|-|49|-|4 +Brand#34|-|LARGE POLISHED NICKEL|-|49|-|4 +Brand#34|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#34|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#34|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#34|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#34|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#34|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#34|-|MEDIUM BRUSHED TIN|-|3|-|4 +Brand#34|-|MEDIUM BRUSHED TIN|-|14|-|4 +Brand#34|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#34|-|MEDIUM BURNISHED STEEL|-|49|-|4 +Brand#34|-|MEDIUM PLATED COPPER|-|36|-|4 +Brand#34|-|MEDIUM PLATED TIN|-|3|-|4 +Brand#34|-|MEDIUM PLATED TIN|-|14|-|4 +Brand#34|-|PROMO ANODIZED COPPER|-|45|-|4 +Brand#34|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#34|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#34|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#34|-|PROMO BRUSHED BRASS|-|9|-|4 +Brand#34|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#34|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#34|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#34|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#34|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#34|-|PROMO PLATED BRASS|-|9|-|4 +Brand#34|-|PROMO PLATED STEEL|-|49|-|4 +Brand#34|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#34|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#34|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#34|-|SMALL ANODIZED BRASS|-|36|-|4 +Brand#34|-|SMALL ANODIZED COPPER|-|45|-|4 +Brand#34|-|SMALL ANODIZED NICKEL|-|14|-|4 +Brand#34|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|3|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#34|-|SMALL ANODIZED STEEL|-|36|-|4 +Brand#34|-|SMALL BRUSHED BRASS|-|14|-|4 +Brand#34|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|14|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|36|-|4 +Brand#34|-|SMALL BRUSHED NICKEL|-|45|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#34|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#34|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#34|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#34|-|SMALL PLATED BRASS|-|14|-|4 +Brand#34|-|SMALL PLATED COPPER|-|36|-|4 +Brand#34|-|SMALL PLATED TIN|-|45|-|4 +Brand#34|-|SMALL POLISHED NICKEL|-|14|-|4 +Brand#34|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#34|-|SMALL POLISHED TIN|-|9|-|4 +Brand#34|-|SMALL POLISHED TIN|-|14|-|4 +Brand#34|-|SMALL POLISHED TIN|-|19|-|4 +Brand#34|-|STANDARD ANODIZED BRASS|-|23|-|4 +Brand#34|-|STANDARD ANODIZED BRASS|-|36|-|4 +Brand#34|-|STANDARD ANODIZED COPPER|-|45|-|4 +Brand#34|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#34|-|STANDARD ANODIZED STEEL|-|9|-|4 +Brand#34|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#34|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#34|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#34|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#34|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#34|-|STANDARD BRUSHED STEEL|-|3|-|4 +Brand#34|-|STANDARD BRUSHED TIN|-|19|-|4 +Brand#34|-|STANDARD BURNISHED COPPER|-|45|-|4 +Brand#34|-|STANDARD BURNISHED NICKEL|-|19|-|4 +Brand#34|-|STANDARD BURNISHED NICKEL|-|45|-|4 +Brand#34|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#34|-|STANDARD BURNISHED TIN|-|45|-|4 +Brand#34|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#34|-|STANDARD PLATED COPPER|-|9|-|4 +Brand#34|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#35|-|ECONOMY ANODIZED COPPER|-|3|-|4 +Brand#35|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#35|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#35|-|ECONOMY BRUSHED NICKEL|-|49|-|4 +Brand#35|-|ECONOMY BRUSHED STEEL|-|23|-|4 +Brand#35|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#35|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#35|-|ECONOMY BRUSHED TIN|-|23|-|4 +Brand#35|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#35|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|9|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#35|-|ECONOMY BURNISHED TIN|-|49|-|4 +Brand#35|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#35|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#35|-|LARGE ANODIZED BRASS|-|3|-|4 +Brand#35|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#35|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#35|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#35|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#35|-|LARGE BRUSHED COPPER|-|9|-|4 +Brand#35|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#35|-|LARGE BRUSHED STEEL|-|3|-|4 +Brand#35|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#35|-|LARGE BURNISHED BRASS|-|36|-|4 +Brand#35|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#35|-|LARGE BURNISHED COPPER|-|23|-|4 +Brand#35|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#35|-|LARGE PLATED BRASS|-|9|-|4 +Brand#35|-|LARGE PLATED COPPER|-|36|-|4 +Brand#35|-|LARGE POLISHED BRASS|-|49|-|4 +Brand#35|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#35|-|LARGE POLISHED TIN|-|14|-|4 +Brand#35|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#35|-|MEDIUM ANODIZED BRASS|-|36|-|4 +Brand#35|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#35|-|MEDIUM BRUSHED BRASS|-|14|-|4 +Brand#35|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#35|-|MEDIUM BRUSHED COPPER|-|36|-|4 +Brand#35|-|MEDIUM BURNISHED BRASS|-|49|-|4 +Brand#35|-|MEDIUM BURNISHED NICKEL|-|45|-|4 +Brand#35|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#35|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#35|-|MEDIUM PLATED COPPER|-|9|-|4 +Brand#35|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#35|-|MEDIUM PLATED NICKEL|-|49|-|4 +Brand#35|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#35|-|PROMO ANODIZED COPPER|-|49|-|4 +Brand#35|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#35|-|PROMO ANODIZED NICKEL|-|23|-|4 +Brand#35|-|PROMO ANODIZED TIN|-|3|-|4 +Brand#35|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#35|-|PROMO BRUSHED BRASS|-|49|-|4 +Brand#35|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#35|-|PROMO BRUSHED NICKEL|-|19|-|4 +Brand#35|-|PROMO BURNISHED BRASS|-|3|-|4 +Brand#35|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#35|-|PROMO PLATED BRASS|-|19|-|4 +Brand#35|-|PROMO PLATED COPPER|-|14|-|4 +Brand#35|-|PROMO PLATED STEEL|-|23|-|4 +Brand#35|-|PROMO PLATED STEEL|-|36|-|4 +Brand#35|-|PROMO PLATED TIN|-|19|-|4 +Brand#35|-|PROMO POLISHED BRASS|-|9|-|4 +Brand#35|-|PROMO POLISHED BRASS|-|36|-|4 +Brand#35|-|PROMO POLISHED NICKEL|-|36|-|4 +Brand#35|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#35|-|PROMO POLISHED TIN|-|36|-|4 +Brand#35|-|PROMO POLISHED TIN|-|45|-|4 +Brand#35|-|SMALL ANODIZED COPPER|-|9|-|4 +Brand#35|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#35|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#35|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#35|-|SMALL BRUSHED STEEL|-|49|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|3|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#35|-|SMALL BRUSHED TIN|-|23|-|4 +Brand#35|-|SMALL BURNISHED BRASS|-|23|-|4 +Brand#35|-|SMALL BURNISHED STEEL|-|36|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#35|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#35|-|SMALL PLATED BRASS|-|23|-|4 +Brand#35|-|SMALL PLATED STEEL|-|14|-|4 +Brand#35|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#35|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#35|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#35|-|SMALL POLISHED TIN|-|23|-|4 +Brand#35|-|SMALL POLISHED TIN|-|45|-|4 +Brand#35|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#35|-|STANDARD ANODIZED STEEL|-|23|-|4 +Brand#35|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#35|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#35|-|STANDARD ANODIZED TIN|-|19|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|3|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#35|-|STANDARD BRUSHED BRASS|-|36|-|4 +Brand#35|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#35|-|STANDARD BRUSHED NICKEL|-|36|-|4 +Brand#35|-|STANDARD BRUSHED NICKEL|-|49|-|4 +Brand#35|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|9|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|19|-|4 +Brand#35|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#35|-|STANDARD BURNISHED COPPER|-|36|-|4 +Brand#35|-|STANDARD BURNISHED STEEL|-|14|-|4 +Brand#35|-|STANDARD PLATED COPPER|-|19|-|4 +Brand#35|-|STANDARD PLATED NICKEL|-|23|-|4 +Brand#35|-|STANDARD PLATED STEEL|-|14|-|4 +Brand#35|-|STANDARD PLATED STEEL|-|23|-|4 +Brand#35|-|STANDARD PLATED TIN|-|49|-|4 +Brand#35|-|STANDARD POLISHED NICKEL|-|23|-|4 +Brand#35|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#35|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#41|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#41|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#41|-|ECONOMY BRUSHED COPPER|-|36|-|4 +Brand#41|-|ECONOMY BRUSHED NICKEL|-|23|-|4 +Brand#41|-|ECONOMY BRUSHED STEEL|-|36|-|4 +Brand#41|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#41|-|ECONOMY BRUSHED TIN|-|14|-|4 +Brand#41|-|ECONOMY PLATED COPPER|-|3|-|4 +Brand#41|-|ECONOMY PLATED STEEL|-|3|-|4 +Brand#41|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#41|-|ECONOMY POLISHED COPPER|-|19|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|14|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|23|-|4 +Brand#41|-|ECONOMY POLISHED NICKEL|-|49|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|9|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|19|-|4 +Brand#41|-|ECONOMY POLISHED STEEL|-|45|-|4 +Brand#41|-|ECONOMY POLISHED TIN|-|19|-|4 +Brand#41|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#41|-|LARGE ANODIZED BRASS|-|23|-|4 +Brand#41|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#41|-|LARGE ANODIZED STEEL|-|3|-|4 +Brand#41|-|LARGE ANODIZED STEEL|-|23|-|4 +Brand#41|-|LARGE BRUSHED COPPER|-|23|-|4 +Brand#41|-|LARGE BRUSHED COPPER|-|49|-|4 +Brand#41|-|LARGE BRUSHED STEEL|-|19|-|4 +Brand#41|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#41|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#41|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#41|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#41|-|LARGE PLATED NICKEL|-|3|-|4 +Brand#41|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#41|-|LARGE PLATED STEEL|-|9|-|4 +Brand#41|-|LARGE PLATED STEEL|-|36|-|4 +Brand#41|-|LARGE PLATED TIN|-|9|-|4 +Brand#41|-|LARGE POLISHED BRASS|-|36|-|4 +Brand#41|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#41|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#41|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#41|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#41|-|MEDIUM ANODIZED BRASS|-|9|-|4 +Brand#41|-|MEDIUM ANODIZED COPPER|-|14|-|4 +Brand#41|-|MEDIUM ANODIZED NICKEL|-|3|-|4 +Brand#41|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#41|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#41|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#41|-|MEDIUM BRUSHED TIN|-|9|-|4 +Brand#41|-|MEDIUM BURNISHED COPPER|-|23|-|4 +Brand#41|-|MEDIUM BURNISHED STEEL|-|9|-|4 +Brand#41|-|MEDIUM BURNISHED STEEL|-|45|-|4 +Brand#41|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#41|-|MEDIUM PLATED BRASS|-|19|-|4 +Brand#41|-|MEDIUM PLATED BRASS|-|45|-|4 +Brand#41|-|MEDIUM PLATED COPPER|-|19|-|4 +Brand#41|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#41|-|MEDIUM PLATED STEEL|-|23|-|4 +Brand#41|-|PROMO ANODIZED BRASS|-|19|-|4 +Brand#41|-|PROMO ANODIZED COPPER|-|9|-|4 +Brand#41|-|PROMO ANODIZED NICKEL|-|9|-|4 +Brand#41|-|PROMO BRUSHED BRASS|-|14|-|4 +Brand#41|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#41|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#41|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#41|-|PROMO BURNISHED NICKEL|-|36|-|4 +Brand#41|-|PROMO BURNISHED TIN|-|3|-|4 +Brand#41|-|PROMO PLATED NICKEL|-|14|-|4 +Brand#41|-|PROMO PLATED NICKEL|-|45|-|4 +Brand#41|-|PROMO PLATED STEEL|-|3|-|4 +Brand#41|-|PROMO PLATED TIN|-|3|-|4 +Brand#41|-|PROMO POLISHED COPPER|-|23|-|4 +Brand#41|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#41|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#41|-|SMALL ANODIZED STEEL|-|45|-|4 +Brand#41|-|SMALL ANODIZED TIN|-|9|-|4 +Brand#41|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#41|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#41|-|SMALL BURNISHED NICKEL|-|3|-|4 +Brand#41|-|SMALL BURNISHED TIN|-|45|-|4 +Brand#41|-|SMALL PLATED COPPER|-|14|-|4 +Brand#41|-|SMALL PLATED COPPER|-|36|-|4 +Brand#41|-|SMALL PLATED COPPER|-|49|-|4 +Brand#41|-|SMALL PLATED TIN|-|19|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|14|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|19|-|4 +Brand#41|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#41|-|SMALL POLISHED TIN|-|45|-|4 +Brand#41|-|STANDARD ANODIZED COPPER|-|19|-|4 +Brand#41|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#41|-|STANDARD ANODIZED STEEL|-|49|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#41|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#41|-|STANDARD BRUSHED BRASS|-|19|-|4 +Brand#41|-|STANDARD BRUSHED NICKEL|-|3|-|4 +Brand#41|-|STANDARD BRUSHED NICKEL|-|9|-|4 +Brand#41|-|STANDARD BRUSHED STEEL|-|45|-|4 +Brand#41|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#41|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#41|-|STANDARD BURNISHED BRASS|-|36|-|4 +Brand#41|-|STANDARD BURNISHED COPPER|-|49|-|4 +Brand#41|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#41|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#41|-|STANDARD PLATED NICKEL|-|14|-|4 +Brand#41|-|STANDARD PLATED STEEL|-|45|-|4 +Brand#41|-|STANDARD PLATED TIN|-|49|-|4 +Brand#41|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#41|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#41|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#42|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#42|-|ECONOMY BRUSHED BRASS|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED COPPER|-|3|-|4 +Brand#42|-|ECONOMY BRUSHED COPPER|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED STEEL|-|14|-|4 +Brand#42|-|ECONOMY BRUSHED TIN|-|19|-|4 +Brand#42|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#42|-|ECONOMY BURNISHED BRASS|-|19|-|4 +Brand#42|-|ECONOMY BURNISHED COPPER|-|23|-|4 +Brand#42|-|ECONOMY BURNISHED NICKEL|-|14|-|4 +Brand#42|-|ECONOMY BURNISHED TIN|-|14|-|4 +Brand#42|-|ECONOMY PLATED COPPER|-|23|-|4 +Brand#42|-|ECONOMY POLISHED BRASS|-|3|-|4 +Brand#42|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#42|-|ECONOMY POLISHED STEEL|-|9|-|4 +Brand#42|-|ECONOMY POLISHED STEEL|-|36|-|4 +Brand#42|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#42|-|LARGE ANODIZED BRASS|-|49|-|4 +Brand#42|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#42|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#42|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#42|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#42|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#42|-|LARGE BRUSHED BRASS|-|49|-|4 +Brand#42|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#42|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#42|-|LARGE BURNISHED COPPER|-|9|-|4 +Brand#42|-|LARGE BURNISHED TIN|-|9|-|4 +Brand#42|-|LARGE PLATED BRASS|-|45|-|4 +Brand#42|-|LARGE PLATED COPPER|-|9|-|4 +Brand#42|-|LARGE PLATED NICKEL|-|36|-|4 +Brand#42|-|LARGE PLATED TIN|-|23|-|4 +Brand#42|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#42|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#42|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#42|-|LARGE POLISHED STEEL|-|9|-|4 +Brand#42|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED COPPER|-|19|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|19|-|4 +Brand#42|-|MEDIUM ANODIZED NICKEL|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|9|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED STEEL|-|23|-|4 +Brand#42|-|MEDIUM ANODIZED TIN|-|14|-|4 +Brand#42|-|MEDIUM ANODIZED TIN|-|19|-|4 +Brand#42|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#42|-|MEDIUM BRUSHED COPPER|-|49|-|4 +Brand#42|-|MEDIUM BRUSHED STEEL|-|36|-|4 +Brand#42|-|MEDIUM BURNISHED COPPER|-|49|-|4 +Brand#42|-|MEDIUM BURNISHED TIN|-|3|-|4 +Brand#42|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#42|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|3|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|23|-|4 +Brand#42|-|MEDIUM PLATED STEEL|-|45|-|4 +Brand#42|-|PROMO ANODIZED NICKEL|-|3|-|4 +Brand#42|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#42|-|PROMO ANODIZED STEEL|-|49|-|4 +Brand#42|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#42|-|PROMO BRUSHED STEEL|-|19|-|4 +Brand#42|-|PROMO BRUSHED TIN|-|45|-|4 +Brand#42|-|PROMO BURNISHED COPPER|-|45|-|4 +Brand#42|-|PROMO BURNISHED NICKEL|-|3|-|4 +Brand#42|-|PROMO BURNISHED STEEL|-|9|-|4 +Brand#42|-|PROMO BURNISHED TIN|-|49|-|4 +Brand#42|-|PROMO PLATED BRASS|-|45|-|4 +Brand#42|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#42|-|PROMO PLATED STEEL|-|19|-|4 +Brand#42|-|PROMO PLATED STEEL|-|45|-|4 +Brand#42|-|PROMO POLISHED COPPER|-|36|-|4 +Brand#42|-|PROMO POLISHED NICKEL|-|3|-|4 +Brand#42|-|SMALL ANODIZED BRASS|-|23|-|4 +Brand#42|-|SMALL ANODIZED COPPER|-|14|-|4 +Brand#42|-|SMALL ANODIZED COPPER|-|19|-|4 +Brand#42|-|SMALL ANODIZED NICKEL|-|23|-|4 +Brand#42|-|SMALL BRUSHED TIN|-|49|-|4 +Brand#42|-|SMALL BURNISHED BRASS|-|3|-|4 +Brand#42|-|SMALL BURNISHED BRASS|-|36|-|4 +Brand#42|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#42|-|SMALL BURNISHED NICKEL|-|9|-|4 +Brand#42|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#42|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#42|-|SMALL PLATED TIN|-|36|-|4 +Brand#42|-|SMALL POLISHED BRASS|-|3|-|4 +Brand#42|-|SMALL POLISHED COPPER|-|36|-|4 +Brand#42|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#42|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#42|-|SMALL POLISHED TIN|-|3|-|4 +Brand#42|-|STANDARD ANODIZED BRASS|-|49|-|4 +Brand#42|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#42|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#42|-|STANDARD ANODIZED NICKEL|-|45|-|4 +Brand#42|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#42|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#42|-|STANDARD BURNISHED STEEL|-|3|-|4 +Brand#42|-|STANDARD BURNISHED TIN|-|19|-|4 +Brand#42|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#42|-|STANDARD PLATED COPPER|-|9|-|4 +Brand#42|-|STANDARD PLATED NICKEL|-|45|-|4 +Brand#42|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#42|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#42|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#42|-|STANDARD POLISHED COPPER|-|14|-|4 +Brand#42|-|STANDARD POLISHED NICKEL|-|45|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|23|-|4 +Brand#42|-|STANDARD POLISHED TIN|-|36|-|4 +Brand#43|-|ECONOMY ANODIZED COPPER|-|19|-|4 +Brand#43|-|ECONOMY ANODIZED COPPER|-|45|-|4 +Brand#43|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#43|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#43|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#43|-|ECONOMY ANODIZED TIN|-|49|-|4 +Brand#43|-|ECONOMY BRUSHED BRASS|-|49|-|4 +Brand#43|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#43|-|ECONOMY BRUSHED NICKEL|-|9|-|4 +Brand#43|-|ECONOMY BURNISHED NICKEL|-|9|-|4 +Brand#43|-|ECONOMY BURNISHED TIN|-|19|-|4 +Brand#43|-|ECONOMY PLATED COPPER|-|36|-|4 +Brand#43|-|ECONOMY PLATED STEEL|-|9|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|19|-|4 +Brand#43|-|ECONOMY PLATED TIN|-|49|-|4 +Brand#43|-|ECONOMY POLISHED COPPER|-|19|-|4 +Brand#43|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#43|-|ECONOMY POLISHED TIN|-|14|-|4 +Brand#43|-|ECONOMY POLISHED TIN|-|45|-|4 +Brand#43|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#43|-|LARGE ANODIZED BRASS|-|36|-|4 +Brand#43|-|LARGE ANODIZED COPPER|-|45|-|4 +Brand#43|-|LARGE BRUSHED COPPER|-|3|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|45|-|4 +Brand#43|-|LARGE BRUSHED NICKEL|-|49|-|4 +Brand#43|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#43|-|LARGE BURNISHED TIN|-|23|-|4 +Brand#43|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#43|-|LARGE PLATED BRASS|-|45|-|4 +Brand#43|-|LARGE PLATED STEEL|-|14|-|4 +Brand#43|-|LARGE PLATED TIN|-|36|-|4 +Brand#43|-|LARGE PLATED TIN|-|45|-|4 +Brand#43|-|LARGE POLISHED BRASS|-|9|-|4 +Brand#43|-|LARGE POLISHED COPPER|-|9|-|4 +Brand#43|-|LARGE POLISHED COPPER|-|19|-|4 +Brand#43|-|LARGE POLISHED STEEL|-|14|-|4 +Brand#43|-|LARGE POLISHED TIN|-|45|-|4 +Brand#43|-|MEDIUM ANODIZED BRASS|-|14|-|4 +Brand#43|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#43|-|MEDIUM ANODIZED COPPER|-|49|-|4 +Brand#43|-|MEDIUM ANODIZED STEEL|-|19|-|4 +Brand#43|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#43|-|MEDIUM BRUSHED BRASS|-|9|-|4 +Brand#43|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#43|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#43|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#43|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#43|-|MEDIUM BURNISHED COPPER|-|14|-|4 +Brand#43|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#43|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#43|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#43|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#43|-|MEDIUM PLATED NICKEL|-|36|-|4 +Brand#43|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#43|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#43|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#43|-|PROMO ANODIZED TIN|-|14|-|4 +Brand#43|-|PROMO BRUSHED NICKEL|-|14|-|4 +Brand#43|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#43|-|PROMO BRUSHED TIN|-|45|-|4 +Brand#43|-|PROMO BURNISHED BRASS|-|49|-|4 +Brand#43|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#43|-|PROMO BURNISHED STEEL|-|3|-|4 +Brand#43|-|PROMO BURNISHED STEEL|-|36|-|4 +Brand#43|-|PROMO BURNISHED TIN|-|36|-|4 +Brand#43|-|PROMO PLATED BRASS|-|19|-|4 +Brand#43|-|PROMO PLATED COPPER|-|45|-|4 +Brand#43|-|PROMO PLATED COPPER|-|49|-|4 +Brand#43|-|PROMO PLATED TIN|-|3|-|4 +Brand#43|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#43|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#43|-|PROMO POLISHED NICKEL|-|49|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|14|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|19|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#43|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#43|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#43|-|SMALL ANODIZED NICKEL|-|9|-|4 +Brand#43|-|SMALL BRUSHED NICKEL|-|3|-|4 +Brand#43|-|SMALL BRUSHED NICKEL|-|9|-|4 +Brand#43|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#43|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#43|-|SMALL PLATED BRASS|-|14|-|4 +Brand#43|-|SMALL PLATED BRASS|-|36|-|4 +Brand#43|-|SMALL PLATED COPPER|-|23|-|4 +Brand#43|-|SMALL PLATED COPPER|-|49|-|4 +Brand#43|-|SMALL PLATED NICKEL|-|36|-|4 +Brand#43|-|SMALL PLATED NICKEL|-|49|-|4 +Brand#43|-|SMALL PLATED STEEL|-|14|-|4 +Brand#43|-|SMALL PLATED TIN|-|49|-|4 +Brand#43|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#43|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#43|-|STANDARD ANODIZED COPPER|-|49|-|4 +Brand#43|-|STANDARD ANODIZED NICKEL|-|14|-|4 +Brand#43|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#43|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#43|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#43|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|9|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|19|-|4 +Brand#43|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#43|-|STANDARD PLATED COPPER|-|36|-|4 +Brand#43|-|STANDARD PLATED NICKEL|-|14|-|4 +Brand#43|-|STANDARD PLATED NICKEL|-|19|-|4 +Brand#43|-|STANDARD PLATED TIN|-|14|-|4 +Brand#43|-|STANDARD POLISHED BRASS|-|23|-|4 +Brand#43|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#44|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#44|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#44|-|ECONOMY ANODIZED NICKEL|-|36|-|4 +Brand#44|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#44|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#44|-|ECONOMY BRUSHED TIN|-|49|-|4 +Brand#44|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#44|-|ECONOMY BURNISHED STEEL|-|45|-|4 +Brand#44|-|ECONOMY PLATED STEEL|-|19|-|4 +Brand#44|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#44|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#44|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#44|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#44|-|ECONOMY POLISHED COPPER|-|45|-|4 +Brand#44|-|ECONOMY POLISHED NICKEL|-|14|-|4 +Brand#44|-|ECONOMY POLISHED NICKEL|-|23|-|4 +Brand#44|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#44|-|ECONOMY POLISHED TIN|-|23|-|4 +Brand#44|-|ECONOMY POLISHED TIN|-|36|-|4 +Brand#44|-|LARGE ANODIZED BRASS|-|19|-|4 +Brand#44|-|LARGE ANODIZED TIN|-|3|-|4 +Brand#44|-|LARGE ANODIZED TIN|-|14|-|4 +Brand#44|-|LARGE BRUSHED TIN|-|3|-|4 +Brand#44|-|LARGE BRUSHED TIN|-|23|-|4 +Brand#44|-|LARGE BURNISHED BRASS|-|23|-|4 +Brand#44|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|3|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#44|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#44|-|LARGE BURNISHED TIN|-|14|-|4 +Brand#44|-|LARGE PLATED BRASS|-|9|-|4 +Brand#44|-|LARGE PLATED BRASS|-|49|-|4 +Brand#44|-|LARGE PLATED NICKEL|-|14|-|4 +Brand#44|-|LARGE PLATED STEEL|-|14|-|4 +Brand#44|-|LARGE PLATED TIN|-|19|-|4 +Brand#44|-|LARGE PLATED TIN|-|23|-|4 +Brand#44|-|LARGE POLISHED STEEL|-|23|-|4 +Brand#44|-|LARGE POLISHED STEEL|-|49|-|4 +Brand#44|-|MEDIUM ANODIZED COPPER|-|45|-|4 +Brand#44|-|MEDIUM ANODIZED NICKEL|-|45|-|4 +Brand#44|-|MEDIUM BRUSHED BRASS|-|49|-|4 +Brand#44|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#44|-|MEDIUM BRUSHED COPPER|-|45|-|4 +Brand#44|-|MEDIUM BRUSHED STEEL|-|19|-|4 +Brand#44|-|MEDIUM BRUSHED TIN|-|49|-|4 +Brand#44|-|MEDIUM BURNISHED COPPER|-|45|-|4 +Brand#44|-|MEDIUM BURNISHED NICKEL|-|23|-|4 +Brand#44|-|MEDIUM BURNISHED TIN|-|23|-|4 +Brand#44|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#44|-|PROMO ANODIZED COPPER|-|23|-|4 +Brand#44|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#44|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#44|-|PROMO BRUSHED COPPER|-|36|-|4 +Brand#44|-|PROMO BRUSHED TIN|-|19|-|4 +Brand#44|-|PROMO PLATED BRASS|-|3|-|4 +Brand#44|-|PROMO PLATED COPPER|-|36|-|4 +Brand#44|-|PROMO PLATED STEEL|-|3|-|4 +Brand#44|-|PROMO PLATED STEEL|-|36|-|4 +Brand#44|-|PROMO PLATED STEEL|-|49|-|4 +Brand#44|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#44|-|PROMO POLISHED BRASS|-|19|-|4 +Brand#44|-|PROMO POLISHED COPPER|-|45|-|4 +Brand#44|-|PROMO POLISHED STEEL|-|36|-|4 +Brand#44|-|PROMO POLISHED TIN|-|9|-|4 +Brand#44|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#44|-|SMALL ANODIZED STEEL|-|23|-|4 +Brand#44|-|SMALL ANODIZED TIN|-|45|-|4 +Brand#44|-|SMALL BRUSHED COPPER|-|14|-|4 +Brand#44|-|SMALL BRUSHED STEEL|-|45|-|4 +Brand#44|-|SMALL BURNISHED COPPER|-|14|-|4 +Brand#44|-|SMALL BURNISHED COPPER|-|49|-|4 +Brand#44|-|SMALL BURNISHED NICKEL|-|14|-|4 +Brand#44|-|SMALL BURNISHED STEEL|-|23|-|4 +Brand#44|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#44|-|SMALL PLATED BRASS|-|36|-|4 +Brand#44|-|SMALL PLATED COPPER|-|19|-|4 +Brand#44|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#44|-|SMALL POLISHED COPPER|-|3|-|4 +Brand#44|-|SMALL POLISHED COPPER|-|49|-|4 +Brand#44|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#44|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#44|-|STANDARD ANODIZED COPPER|-|3|-|4 +Brand#44|-|STANDARD ANODIZED NICKEL|-|3|-|4 +Brand#44|-|STANDARD ANODIZED NICKEL|-|36|-|4 +Brand#44|-|STANDARD ANODIZED STEEL|-|14|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|9|-|4 +Brand#44|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#44|-|STANDARD BRUSHED COPPER|-|36|-|4 +Brand#44|-|STANDARD BRUSHED COPPER|-|45|-|4 +Brand#44|-|STANDARD BRUSHED TIN|-|9|-|4 +Brand#44|-|STANDARD BRUSHED TIN|-|49|-|4 +Brand#44|-|STANDARD BURNISHED COPPER|-|9|-|4 +Brand#44|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|14|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|23|-|4 +Brand#44|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#44|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#44|-|STANDARD POLISHED NICKEL|-|19|-|4 +Brand#44|-|STANDARD POLISHED TIN|-|9|-|4 +Brand#51|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#51|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#51|-|ECONOMY ANODIZED NICKEL|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|23|-|4 +Brand#51|-|ECONOMY ANODIZED STEEL|-|49|-|4 +Brand#51|-|ECONOMY BRUSHED BRASS|-|3|-|4 +Brand#51|-|ECONOMY BRUSHED BRASS|-|49|-|4 +Brand#51|-|ECONOMY BRUSHED NICKEL|-|14|-|4 +Brand#51|-|ECONOMY BRUSHED STEEL|-|45|-|4 +Brand#51|-|ECONOMY BRUSHED TIN|-|36|-|4 +Brand#51|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#51|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#51|-|ECONOMY PLATED NICKEL|-|49|-|4 +Brand#51|-|ECONOMY PLATED TIN|-|36|-|4 +Brand#51|-|ECONOMY POLISHED COPPER|-|9|-|4 +Brand#51|-|ECONOMY POLISHED STEEL|-|14|-|4 +Brand#51|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#51|-|LARGE ANODIZED COPPER|-|9|-|4 +Brand#51|-|LARGE ANODIZED COPPER|-|49|-|4 +Brand#51|-|LARGE ANODIZED NICKEL|-|14|-|4 +Brand#51|-|LARGE ANODIZED STEEL|-|36|-|4 +Brand#51|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#51|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#51|-|LARGE BURNISHED BRASS|-|19|-|4 +Brand#51|-|LARGE BURNISHED BRASS|-|36|-|4 +Brand#51|-|LARGE BURNISHED COPPER|-|14|-|4 +Brand#51|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#51|-|LARGE PLATED BRASS|-|36|-|4 +Brand#51|-|LARGE POLISHED COPPER|-|14|-|4 +Brand#51|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#51|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#51|-|LARGE POLISHED STEEL|-|19|-|4 +Brand#51|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#51|-|MEDIUM ANODIZED STEEL|-|3|-|4 +Brand#51|-|MEDIUM BRUSHED BRASS|-|36|-|4 +Brand#51|-|MEDIUM BRUSHED BRASS|-|45|-|4 +Brand#51|-|MEDIUM BRUSHED STEEL|-|3|-|4 +Brand#51|-|MEDIUM BRUSHED TIN|-|36|-|4 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|3|-|4 +Brand#51|-|MEDIUM BURNISHED NICKEL|-|36|-|4 +Brand#51|-|MEDIUM BURNISHED STEEL|-|14|-|4 +Brand#51|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#51|-|MEDIUM PLATED STEEL|-|19|-|4 +Brand#51|-|MEDIUM PLATED TIN|-|3|-|4 +Brand#51|-|PROMO ANODIZED NICKEL|-|14|-|4 +Brand#51|-|PROMO ANODIZED STEEL|-|23|-|4 +Brand#51|-|PROMO ANODIZED TIN|-|19|-|4 +Brand#51|-|PROMO BRUSHED BRASS|-|23|-|4 +Brand#51|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#51|-|PROMO BRUSHED STEEL|-|45|-|4 +Brand#51|-|PROMO BRUSHED TIN|-|9|-|4 +Brand#51|-|PROMO BURNISHED BRASS|-|19|-|4 +Brand#51|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#51|-|PROMO BURNISHED NICKEL|-|14|-|4 +Brand#51|-|PROMO PLATED BRASS|-|3|-|4 +Brand#51|-|PROMO PLATED BRASS|-|23|-|4 +Brand#51|-|PROMO PLATED TIN|-|19|-|4 +Brand#51|-|PROMO PLATED TIN|-|23|-|4 +Brand#51|-|PROMO POLISHED BRASS|-|23|-|4 +Brand#51|-|PROMO POLISHED COPPER|-|9|-|4 +Brand#51|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#51|-|PROMO POLISHED STEEL|-|49|-|4 +Brand#51|-|SMALL ANODIZED STEEL|-|14|-|4 +Brand#51|-|SMALL BRUSHED BRASS|-|23|-|4 +Brand#51|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#51|-|SMALL BURNISHED NICKEL|-|23|-|4 +Brand#51|-|SMALL PLATED COPPER|-|49|-|4 +Brand#51|-|SMALL PLATED NICKEL|-|3|-|4 +Brand#51|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#51|-|SMALL PLATED STEEL|-|45|-|4 +Brand#51|-|SMALL POLISHED NICKEL|-|14|-|4 +Brand#51|-|SMALL POLISHED NICKEL|-|23|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|3|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#51|-|SMALL POLISHED STEEL|-|49|-|4 +Brand#51|-|STANDARD ANODIZED NICKEL|-|3|-|4 +Brand#51|-|STANDARD ANODIZED NICKEL|-|49|-|4 +Brand#51|-|STANDARD BRUSHED BRASS|-|3|-|4 +Brand#51|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#51|-|STANDARD BRUSHED NICKEL|-|19|-|4 +Brand#51|-|STANDARD BRUSHED STEEL|-|36|-|4 +Brand#51|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#51|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#51|-|STANDARD BURNISHED STEEL|-|23|-|4 +Brand#51|-|STANDARD BURNISHED STEEL|-|36|-|4 +Brand#51|-|STANDARD BURNISHED TIN|-|45|-|4 +Brand#51|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#51|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#51|-|STANDARD PLATED COPPER|-|14|-|4 +Brand#51|-|STANDARD PLATED COPPER|-|23|-|4 +Brand#51|-|STANDARD POLISHED BRASS|-|14|-|4 +Brand#51|-|STANDARD POLISHED BRASS|-|45|-|4 +Brand#51|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#51|-|STANDARD POLISHED STEEL|-|49|-|4 +Brand#51|-|STANDARD POLISHED TIN|-|45|-|4 +Brand#52|-|ECONOMY ANODIZED BRASS|-|14|-|4 +Brand#52|-|ECONOMY ANODIZED BRASS|-|23|-|4 +Brand#52|-|ECONOMY ANODIZED COPPER|-|36|-|4 +Brand#52|-|ECONOMY ANODIZED NICKEL|-|49|-|4 +Brand#52|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#52|-|ECONOMY BRUSHED COPPER|-|49|-|4 +Brand#52|-|ECONOMY BURNISHED BRASS|-|36|-|4 +Brand#52|-|ECONOMY BURNISHED COPPER|-|19|-|4 +Brand#52|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#52|-|ECONOMY BURNISHED NICKEL|-|19|-|4 +Brand#52|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#52|-|ECONOMY PLATED TIN|-|14|-|4 +Brand#52|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#52|-|ECONOMY POLISHED BRASS|-|23|-|4 +Brand#52|-|ECONOMY POLISHED BRASS|-|45|-|4 +Brand#52|-|ECONOMY POLISHED NICKEL|-|36|-|4 +Brand#52|-|ECONOMY POLISHED STEEL|-|49|-|4 +Brand#52|-|LARGE ANODIZED COPPER|-|14|-|4 +Brand#52|-|LARGE ANODIZED NICKEL|-|3|-|4 +Brand#52|-|LARGE ANODIZED NICKEL|-|45|-|4 +Brand#52|-|LARGE ANODIZED TIN|-|45|-|4 +Brand#52|-|LARGE BRUSHED COPPER|-|19|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|3|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|19|-|4 +Brand#52|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#52|-|LARGE BRUSHED STEEL|-|49|-|4 +Brand#52|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#52|-|LARGE BURNISHED NICKEL|-|9|-|4 +Brand#52|-|LARGE BURNISHED TIN|-|23|-|4 +Brand#52|-|LARGE BURNISHED TIN|-|45|-|4 +Brand#52|-|LARGE PLATED BRASS|-|14|-|4 +Brand#52|-|LARGE PLATED COPPER|-|14|-|4 +Brand#52|-|LARGE PLATED COPPER|-|19|-|4 +Brand#52|-|LARGE PLATED NICKEL|-|45|-|4 +Brand#52|-|LARGE PLATED STEEL|-|9|-|4 +Brand#52|-|LARGE PLATED TIN|-|9|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|19|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|23|-|4 +Brand#52|-|LARGE POLISHED NICKEL|-|36|-|4 +Brand#52|-|LARGE POLISHED TIN|-|9|-|4 +Brand#52|-|MEDIUM ANODIZED COPPER|-|36|-|4 +Brand#52|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#52|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#52|-|MEDIUM ANODIZED TIN|-|49|-|4 +Brand#52|-|MEDIUM BRUSHED COPPER|-|9|-|4 +Brand#52|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#52|-|MEDIUM BRUSHED STEEL|-|23|-|4 +Brand#52|-|MEDIUM BRUSHED STEEL|-|49|-|4 +Brand#52|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#52|-|MEDIUM BURNISHED TIN|-|45|-|4 +Brand#52|-|MEDIUM BURNISHED TIN|-|49|-|4 +Brand#52|-|MEDIUM PLATED BRASS|-|36|-|4 +Brand#52|-|MEDIUM PLATED STEEL|-|9|-|4 +Brand#52|-|MEDIUM PLATED STEEL|-|49|-|4 +Brand#52|-|MEDIUM PLATED TIN|-|9|-|4 +Brand#52|-|MEDIUM PLATED TIN|-|49|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|9|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|23|-|4 +Brand#52|-|PROMO ANODIZED BRASS|-|36|-|4 +Brand#52|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#52|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#52|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#52|-|PROMO BRUSHED NICKEL|-|3|-|4 +Brand#52|-|PROMO BRUSHED NICKEL|-|49|-|4 +Brand#52|-|PROMO BRUSHED STEEL|-|14|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|3|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|19|-|4 +Brand#52|-|PROMO BRUSHED TIN|-|36|-|4 +Brand#52|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#52|-|PROMO BURNISHED NICKEL|-|9|-|4 +Brand#52|-|PROMO BURNISHED STEEL|-|9|-|4 +Brand#52|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#52|-|PROMO BURNISHED TIN|-|19|-|4 +Brand#52|-|PROMO BURNISHED TIN|-|36|-|4 +Brand#52|-|PROMO PLATED BRASS|-|19|-|4 +Brand#52|-|PROMO PLATED BRASS|-|45|-|4 +Brand#52|-|PROMO PLATED BRASS|-|49|-|4 +Brand#52|-|PROMO PLATED COPPER|-|9|-|4 +Brand#52|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#52|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#52|-|PROMO POLISHED NICKEL|-|14|-|4 +Brand#52|-|PROMO POLISHED NICKEL|-|49|-|4 +Brand#52|-|PROMO POLISHED TIN|-|36|-|4 +Brand#52|-|SMALL ANODIZED BRASS|-|3|-|4 +Brand#52|-|SMALL ANODIZED BRASS|-|14|-|4 +Brand#52|-|SMALL ANODIZED COPPER|-|3|-|4 +Brand#52|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#52|-|SMALL ANODIZED STEEL|-|9|-|4 +Brand#52|-|SMALL ANODIZED STEEL|-|19|-|4 +Brand#52|-|SMALL BRUSHED NICKEL|-|19|-|4 +Brand#52|-|SMALL BRUSHED STEEL|-|23|-|4 +Brand#52|-|SMALL BRUSHED TIN|-|14|-|4 +Brand#52|-|SMALL BRUSHED TIN|-|19|-|4 +Brand#52|-|SMALL BURNISHED NICKEL|-|14|-|4 +Brand#52|-|SMALL BURNISHED NICKEL|-|49|-|4 +Brand#52|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#52|-|SMALL POLISHED BRASS|-|36|-|4 +Brand#52|-|SMALL POLISHED BRASS|-|49|-|4 +Brand#52|-|SMALL POLISHED TIN|-|45|-|4 +Brand#52|-|STANDARD ANODIZED BRASS|-|45|-|4 +Brand#52|-|STANDARD BRUSHED BRASS|-|23|-|4 +Brand#52|-|STANDARD BRUSHED COPPER|-|14|-|4 +Brand#52|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#52|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#52|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#52|-|STANDARD BURNISHED TIN|-|9|-|4 +Brand#52|-|STANDARD BURNISHED TIN|-|19|-|4 +Brand#52|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#52|-|STANDARD PLATED STEEL|-|36|-|4 +Brand#52|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#52|-|STANDARD POLISHED COPPER|-|45|-|4 +Brand#52|-|STANDARD POLISHED STEEL|-|19|-|4 +Brand#52|-|STANDARD POLISHED TIN|-|19|-|4 +Brand#53|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#53|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#53|-|ECONOMY ANODIZED NICKEL|-|3|-|4 +Brand#53|-|ECONOMY ANODIZED NICKEL|-|19|-|4 +Brand#53|-|ECONOMY ANODIZED STEEL|-|45|-|4 +Brand#53|-|ECONOMY ANODIZED TIN|-|14|-|4 +Brand#53|-|ECONOMY ANODIZED TIN|-|36|-|4 +Brand#53|-|ECONOMY BRUSHED TIN|-|45|-|4 +Brand#53|-|ECONOMY BURNISHED BRASS|-|14|-|4 +Brand#53|-|ECONOMY BURNISHED COPPER|-|45|-|4 +Brand#53|-|ECONOMY BURNISHED NICKEL|-|3|-|4 +Brand#53|-|ECONOMY BURNISHED NICKEL|-|49|-|4 +Brand#53|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#53|-|ECONOMY PLATED BRASS|-|3|-|4 +Brand#53|-|ECONOMY PLATED NICKEL|-|14|-|4 +Brand#53|-|ECONOMY PLATED STEEL|-|23|-|4 +Brand#53|-|ECONOMY PLATED STEEL|-|36|-|4 +Brand#53|-|ECONOMY POLISHED TIN|-|36|-|4 +Brand#53|-|LARGE ANODIZED NICKEL|-|49|-|4 +Brand#53|-|LARGE ANODIZED STEEL|-|19|-|4 +Brand#53|-|LARGE BRUSHED COPPER|-|3|-|4 +Brand#53|-|LARGE BRUSHED COPPER|-|14|-|4 +Brand#53|-|LARGE BRUSHED NICKEL|-|23|-|4 +Brand#53|-|LARGE BRUSHED NICKEL|-|36|-|4 +Brand#53|-|LARGE BRUSHED TIN|-|36|-|4 +Brand#53|-|LARGE BURNISHED BRASS|-|45|-|4 +Brand#53|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#53|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#53|-|LARGE BURNISHED NICKEL|-|23|-|4 +Brand#53|-|LARGE BURNISHED STEEL|-|19|-|4 +Brand#53|-|LARGE BURNISHED STEEL|-|23|-|4 +Brand#53|-|LARGE PLATED BRASS|-|9|-|4 +Brand#53|-|LARGE PLATED BRASS|-|45|-|4 +Brand#53|-|LARGE PLATED BRASS|-|49|-|4 +Brand#53|-|LARGE PLATED COPPER|-|23|-|4 +Brand#53|-|LARGE PLATED NICKEL|-|23|-|4 +Brand#53|-|LARGE PLATED NICKEL|-|49|-|4 +Brand#53|-|LARGE PLATED STEEL|-|49|-|4 +Brand#53|-|LARGE PLATED TIN|-|14|-|4 +Brand#53|-|LARGE POLISHED COPPER|-|49|-|4 +Brand#53|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#53|-|LARGE POLISHED TIN|-|9|-|4 +Brand#53|-|MEDIUM ANODIZED BRASS|-|23|-|4 +Brand#53|-|MEDIUM ANODIZED STEEL|-|14|-|4 +Brand#53|-|MEDIUM ANODIZED STEEL|-|36|-|4 +Brand#53|-|MEDIUM ANODIZED TIN|-|3|-|4 +Brand#53|-|MEDIUM ANODIZED TIN|-|9|-|4 +Brand#53|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#53|-|MEDIUM BRUSHED COPPER|-|3|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|14|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|36|-|4 +Brand#53|-|MEDIUM BRUSHED NICKEL|-|49|-|4 +Brand#53|-|MEDIUM BRUSHED STEEL|-|45|-|4 +Brand#53|-|MEDIUM BURNISHED BRASS|-|3|-|4 +Brand#53|-|MEDIUM BURNISHED BRASS|-|36|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|9|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|14|-|4 +Brand#53|-|MEDIUM BURNISHED TIN|-|36|-|4 +Brand#53|-|MEDIUM PLATED BRASS|-|23|-|4 +Brand#53|-|MEDIUM PLATED COPPER|-|14|-|4 +Brand#53|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#53|-|MEDIUM PLATED TIN|-|19|-|4 +Brand#53|-|MEDIUM PLATED TIN|-|45|-|4 +Brand#53|-|PROMO ANODIZED BRASS|-|36|-|4 +Brand#53|-|PROMO ANODIZED NICKEL|-|3|-|4 +Brand#53|-|PROMO ANODIZED NICKEL|-|19|-|4 +Brand#53|-|PROMO BRUSHED BRASS|-|45|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|3|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|23|-|4 +Brand#53|-|PROMO BRUSHED COPPER|-|45|-|4 +Brand#53|-|PROMO BURNISHED BRASS|-|23|-|4 +Brand#53|-|PROMO BURNISHED BRASS|-|36|-|4 +Brand#53|-|PROMO BURNISHED NICKEL|-|23|-|4 +Brand#53|-|PROMO BURNISHED STEEL|-|23|-|4 +Brand#53|-|PROMO BURNISHED STEEL|-|49|-|4 +Brand#53|-|PROMO PLATED TIN|-|19|-|4 +Brand#53|-|PROMO PLATED TIN|-|23|-|4 +Brand#53|-|PROMO PLATED TIN|-|36|-|4 +Brand#53|-|PROMO POLISHED STEEL|-|23|-|4 +Brand#53|-|PROMO POLISHED TIN|-|3|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|23|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|36|-|4 +Brand#53|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#53|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#53|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#53|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#53|-|SMALL BRUSHED TIN|-|3|-|4 +Brand#53|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#53|-|SMALL BURNISHED BRASS|-|9|-|4 +Brand#53|-|SMALL BURNISHED BRASS|-|49|-|4 +Brand#53|-|SMALL BURNISHED COPPER|-|19|-|4 +Brand#53|-|SMALL BURNISHED COPPER|-|45|-|4 +Brand#53|-|SMALL PLATED BRASS|-|9|-|4 +Brand#53|-|SMALL PLATED COPPER|-|3|-|4 +Brand#53|-|SMALL PLATED NICKEL|-|14|-|4 +Brand#53|-|SMALL POLISHED NICKEL|-|19|-|4 +Brand#53|-|SMALL POLISHED STEEL|-|36|-|4 +Brand#53|-|SMALL POLISHED TIN|-|23|-|4 +Brand#53|-|STANDARD ANODIZED BRASS|-|14|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|9|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|23|-|4 +Brand#53|-|STANDARD ANODIZED NICKEL|-|45|-|4 +Brand#53|-|STANDARD ANODIZED STEEL|-|45|-|4 +Brand#53|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#53|-|STANDARD BRUSHED NICKEL|-|23|-|4 +Brand#53|-|STANDARD BRUSHED TIN|-|14|-|4 +Brand#53|-|STANDARD BURNISHED NICKEL|-|49|-|4 +Brand#53|-|STANDARD BURNISHED STEEL|-|9|-|4 +Brand#53|-|STANDARD PLATED BRASS|-|36|-|4 +Brand#53|-|STANDARD PLATED COPPER|-|45|-|4 +Brand#53|-|STANDARD PLATED NICKEL|-|36|-|4 +Brand#53|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#53|-|STANDARD PLATED STEEL|-|49|-|4 +Brand#53|-|STANDARD PLATED TIN|-|23|-|4 +Brand#53|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#54|-|ECONOMY ANODIZED BRASS|-|9|-|4 +Brand#54|-|ECONOMY ANODIZED BRASS|-|45|-|4 +Brand#54|-|ECONOMY ANODIZED COPPER|-|9|-|4 +Brand#54|-|ECONOMY ANODIZED STEEL|-|19|-|4 +Brand#54|-|ECONOMY BRUSHED BRASS|-|45|-|4 +Brand#54|-|ECONOMY BRUSHED NICKEL|-|19|-|4 +Brand#54|-|ECONOMY BRUSHED STEEL|-|3|-|4 +Brand#54|-|ECONOMY BRUSHED TIN|-|19|-|4 +Brand#54|-|ECONOMY BURNISHED BRASS|-|45|-|4 +Brand#54|-|ECONOMY BURNISHED COPPER|-|14|-|4 +Brand#54|-|ECONOMY BURNISHED NICKEL|-|9|-|4 +Brand#54|-|ECONOMY BURNISHED NICKEL|-|36|-|4 +Brand#54|-|ECONOMY BURNISHED STEEL|-|36|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|9|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|14|-|4 +Brand#54|-|ECONOMY BURNISHED TIN|-|23|-|4 +Brand#54|-|ECONOMY PLATED TIN|-|23|-|4 +Brand#54|-|ECONOMY POLISHED BRASS|-|9|-|4 +Brand#54|-|ECONOMY POLISHED BRASS|-|19|-|4 +Brand#54|-|ECONOMY POLISHED COPPER|-|23|-|4 +Brand#54|-|ECONOMY POLISHED STEEL|-|23|-|4 +Brand#54|-|ECONOMY POLISHED TIN|-|3|-|4 +Brand#54|-|LARGE ANODIZED BRASS|-|14|-|4 +Brand#54|-|LARGE ANODIZED BRASS|-|49|-|4 +Brand#54|-|LARGE ANODIZED TIN|-|9|-|4 +Brand#54|-|LARGE BRUSHED BRASS|-|14|-|4 +Brand#54|-|LARGE BRUSHED STEEL|-|9|-|4 +Brand#54|-|LARGE BRUSHED STEEL|-|23|-|4 +Brand#54|-|LARGE BRUSHED TIN|-|14|-|4 +Brand#54|-|LARGE BURNISHED BRASS|-|49|-|4 +Brand#54|-|LARGE BURNISHED COPPER|-|19|-|4 +Brand#54|-|LARGE BURNISHED NICKEL|-|14|-|4 +Brand#54|-|LARGE BURNISHED TIN|-|14|-|4 +Brand#54|-|LARGE PLATED BRASS|-|19|-|4 +Brand#54|-|LARGE PLATED BRASS|-|23|-|4 +Brand#54|-|LARGE POLISHED BRASS|-|19|-|4 +Brand#54|-|LARGE POLISHED BRASS|-|23|-|4 +Brand#54|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#54|-|LARGE POLISHED NICKEL|-|14|-|4 +Brand#54|-|LARGE POLISHED STEEL|-|19|-|4 +Brand#54|-|LARGE POLISHED TIN|-|3|-|4 +Brand#54|-|LARGE POLISHED TIN|-|9|-|4 +Brand#54|-|LARGE POLISHED TIN|-|36|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|9|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|14|-|4 +Brand#54|-|MEDIUM ANODIZED NICKEL|-|36|-|4 +Brand#54|-|MEDIUM BRUSHED NICKEL|-|9|-|4 +Brand#54|-|MEDIUM BRUSHED NICKEL|-|19|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|3|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|19|-|4 +Brand#54|-|MEDIUM BURNISHED STEEL|-|23|-|4 +Brand#54|-|MEDIUM PLATED BRASS|-|3|-|4 +Brand#54|-|MEDIUM PLATED NICKEL|-|45|-|4 +Brand#54|-|PROMO ANODIZED NICKEL|-|45|-|4 +Brand#54|-|PROMO BRUSHED BRASS|-|3|-|4 +Brand#54|-|PROMO BRUSHED STEEL|-|23|-|4 +Brand#54|-|PROMO BRUSHED TIN|-|14|-|4 +Brand#54|-|PROMO BURNISHED COPPER|-|49|-|4 +Brand#54|-|PROMO BURNISHED TIN|-|9|-|4 +Brand#54|-|PROMO PLATED BRASS|-|14|-|4 +Brand#54|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#54|-|PROMO PLATED STEEL|-|19|-|4 +Brand#54|-|PROMO PLATED TIN|-|23|-|4 +Brand#54|-|PROMO PLATED TIN|-|49|-|4 +Brand#54|-|PROMO POLISHED BRASS|-|3|-|4 +Brand#54|-|PROMO POLISHED NICKEL|-|9|-|4 +Brand#54|-|PROMO POLISHED TIN|-|49|-|4 +Brand#54|-|SMALL ANODIZED COPPER|-|49|-|4 +Brand#54|-|SMALL ANODIZED NICKEL|-|9|-|4 +Brand#54|-|SMALL ANODIZED NICKEL|-|36|-|4 +Brand#54|-|SMALL ANODIZED TIN|-|19|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|14|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|19|-|4 +Brand#54|-|SMALL BRUSHED BRASS|-|36|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|3|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|9|-|4 +Brand#54|-|SMALL BRUSHED COPPER|-|19|-|4 +Brand#54|-|SMALL BRUSHED TIN|-|9|-|4 +Brand#54|-|SMALL BRUSHED TIN|-|36|-|4 +Brand#54|-|SMALL BURNISHED COPPER|-|9|-|4 +Brand#54|-|SMALL BURNISHED COPPER|-|36|-|4 +Brand#54|-|SMALL BURNISHED STEEL|-|14|-|4 +Brand#54|-|SMALL BURNISHED STEEL|-|19|-|4 +Brand#54|-|SMALL BURNISHED TIN|-|9|-|4 +Brand#54|-|SMALL BURNISHED TIN|-|36|-|4 +Brand#54|-|SMALL PLATED BRASS|-|23|-|4 +Brand#54|-|SMALL PLATED COPPER|-|9|-|4 +Brand#54|-|SMALL PLATED COPPER|-|36|-|4 +Brand#54|-|SMALL PLATED COPPER|-|49|-|4 +Brand#54|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#54|-|SMALL PLATED TIN|-|23|-|4 +Brand#54|-|SMALL PLATED TIN|-|36|-|4 +Brand#54|-|SMALL POLISHED BRASS|-|9|-|4 +Brand#54|-|SMALL POLISHED COPPER|-|9|-|4 +Brand#54|-|SMALL POLISHED TIN|-|9|-|4 +Brand#54|-|STANDARD ANODIZED BRASS|-|3|-|4 +Brand#54|-|STANDARD ANODIZED BRASS|-|9|-|4 +Brand#54|-|STANDARD ANODIZED COPPER|-|3|-|4 +Brand#54|-|STANDARD ANODIZED TIN|-|3|-|4 +Brand#54|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#54|-|STANDARD BRUSHED NICKEL|-|45|-|4 +Brand#54|-|STANDARD BRUSHED TIN|-|36|-|4 +Brand#54|-|STANDARD BURNISHED BRASS|-|23|-|4 +Brand#54|-|STANDARD BURNISHED BRASS|-|49|-|4 +Brand#54|-|STANDARD BURNISHED COPPER|-|19|-|4 +Brand#54|-|STANDARD BURNISHED NICKEL|-|23|-|4 +Brand#54|-|STANDARD BURNISHED STEEL|-|45|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|3|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|45|-|4 +Brand#54|-|STANDARD PLATED BRASS|-|49|-|4 +Brand#54|-|STANDARD PLATED STEEL|-|3|-|4 +Brand#54|-|STANDARD POLISHED BRASS|-|36|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|3|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|14|-|4 +Brand#54|-|STANDARD POLISHED STEEL|-|45|-|4 +Brand#55|-|ECONOMY ANODIZED BRASS|-|3|-|4 +Brand#55|-|ECONOMY BRUSHED BRASS|-|19|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|9|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|23|-|4 +Brand#55|-|ECONOMY BRUSHED COPPER|-|45|-|4 +Brand#55|-|ECONOMY BRUSHED STEEL|-|23|-|4 +Brand#55|-|ECONOMY BURNISHED NICKEL|-|36|-|4 +Brand#55|-|ECONOMY BURNISHED NICKEL|-|45|-|4 +Brand#55|-|ECONOMY BURNISHED TIN|-|45|-|4 +Brand#55|-|ECONOMY PLATED NICKEL|-|19|-|4 +Brand#55|-|ECONOMY POLISHED NICKEL|-|9|-|4 +Brand#55|-|LARGE BRUSHED BRASS|-|23|-|4 +Brand#55|-|LARGE BRUSHED BRASS|-|45|-|4 +Brand#55|-|LARGE BRUSHED COPPER|-|49|-|4 +Brand#55|-|LARGE BRUSHED NICKEL|-|9|-|4 +Brand#55|-|LARGE BRUSHED NICKEL|-|14|-|4 +Brand#55|-|LARGE BURNISHED BRASS|-|3|-|4 +Brand#55|-|LARGE BURNISHED COPPER|-|14|-|4 +Brand#55|-|LARGE BURNISHED COPPER|-|36|-|4 +Brand#55|-|LARGE PLATED BRASS|-|45|-|4 +Brand#55|-|LARGE PLATED COPPER|-|19|-|4 +Brand#55|-|LARGE PLATED NICKEL|-|9|-|4 +Brand#55|-|LARGE PLATED STEEL|-|9|-|4 +Brand#55|-|LARGE PLATED TIN|-|9|-|4 +Brand#55|-|LARGE PLATED TIN|-|14|-|4 +Brand#55|-|LARGE PLATED TIN|-|23|-|4 +Brand#55|-|LARGE POLISHED NICKEL|-|3|-|4 +Brand#55|-|LARGE POLISHED STEEL|-|36|-|4 +Brand#55|-|LARGE POLISHED STEEL|-|45|-|4 +Brand#55|-|MEDIUM ANODIZED COPPER|-|9|-|4 +Brand#55|-|MEDIUM BRUSHED BRASS|-|3|-|4 +Brand#55|-|MEDIUM BRUSHED NICKEL|-|23|-|4 +Brand#55|-|MEDIUM BRUSHED TIN|-|45|-|4 +Brand#55|-|MEDIUM BURNISHED BRASS|-|23|-|4 +Brand#55|-|MEDIUM BURNISHED COPPER|-|36|-|4 +Brand#55|-|MEDIUM BURNISHED NICKEL|-|3|-|4 +Brand#55|-|MEDIUM BURNISHED STEEL|-|14|-|4 +Brand#55|-|MEDIUM BURNISHED STEEL|-|36|-|4 +Brand#55|-|MEDIUM PLATED NICKEL|-|23|-|4 +Brand#55|-|PROMO ANODIZED COPPER|-|14|-|4 +Brand#55|-|PROMO ANODIZED COPPER|-|49|-|4 +Brand#55|-|PROMO ANODIZED STEEL|-|36|-|4 +Brand#55|-|PROMO ANODIZED TIN|-|23|-|4 +Brand#55|-|PROMO BRUSHED NICKEL|-|36|-|4 +Brand#55|-|PROMO BRUSHED STEEL|-|3|-|4 +Brand#55|-|PROMO BRUSHED STEEL|-|36|-|4 +Brand#55|-|PROMO BRUSHED TIN|-|9|-|4 +Brand#55|-|PROMO BURNISHED COPPER|-|3|-|4 +Brand#55|-|PROMO BURNISHED STEEL|-|14|-|4 +Brand#55|-|PROMO BURNISHED TIN|-|23|-|4 +Brand#55|-|PROMO BURNISHED TIN|-|49|-|4 +Brand#55|-|PROMO PLATED COPPER|-|3|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|3|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|14|-|4 +Brand#55|-|PROMO PLATED NICKEL|-|23|-|4 +Brand#55|-|PROMO PLATED TIN|-|3|-|4 +Brand#55|-|PROMO POLISHED COPPER|-|3|-|4 +Brand#55|-|SMALL ANODIZED BRASS|-|19|-|4 +Brand#55|-|SMALL ANODIZED NICKEL|-|45|-|4 +Brand#55|-|SMALL BRUSHED COPPER|-|14|-|4 +Brand#55|-|SMALL BRUSHED COPPER|-|45|-|4 +Brand#55|-|SMALL BURNISHED BRASS|-|14|-|4 +Brand#55|-|SMALL BURNISHED TIN|-|3|-|4 +Brand#55|-|SMALL BURNISHED TIN|-|49|-|4 +Brand#55|-|SMALL PLATED BRASS|-|45|-|4 +Brand#55|-|SMALL PLATED COPPER|-|23|-|4 +Brand#55|-|SMALL PLATED COPPER|-|36|-|4 +Brand#55|-|SMALL PLATED COPPER|-|45|-|4 +Brand#55|-|SMALL PLATED COPPER|-|49|-|4 +Brand#55|-|SMALL PLATED NICKEL|-|9|-|4 +Brand#55|-|SMALL PLATED STEEL|-|9|-|4 +Brand#55|-|SMALL PLATED TIN|-|14|-|4 +Brand#55|-|SMALL PLATED TIN|-|36|-|4 +Brand#55|-|SMALL POLISHED NICKEL|-|45|-|4 +Brand#55|-|SMALL POLISHED STEEL|-|19|-|4 +Brand#55|-|SMALL POLISHED TIN|-|19|-|4 +Brand#55|-|STANDARD ANODIZED BRASS|-|36|-|4 +Brand#55|-|STANDARD ANODIZED BRASS|-|49|-|4 +Brand#55|-|STANDARD ANODIZED STEEL|-|19|-|4 +Brand#55|-|STANDARD ANODIZED TIN|-|36|-|4 +Brand#55|-|STANDARD ANODIZED TIN|-|49|-|4 +Brand#55|-|STANDARD BRUSHED BRASS|-|36|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|3|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|9|-|4 +Brand#55|-|STANDARD BRUSHED COPPER|-|23|-|4 +Brand#55|-|STANDARD BRUSHED STEEL|-|19|-|4 +Brand#55|-|STANDARD BRUSHED TIN|-|23|-|4 +Brand#55|-|STANDARD BRUSHED TIN|-|45|-|4 +Brand#55|-|STANDARD BURNISHED BRASS|-|19|-|4 +Brand#55|-|STANDARD BURNISHED NICKEL|-|3|-|4 +Brand#55|-|STANDARD BURNISHED NICKEL|-|36|-|4 +Brand#55|-|STANDARD BURNISHED STEEL|-|19|-|4 +Brand#55|-|STANDARD PLATED BRASS|-|23|-|4 +Brand#55|-|STANDARD PLATED NICKEL|-|9|-|4 +Brand#55|-|STANDARD PLATED TIN|-|36|-|4 +Brand#55|-|STANDARD POLISHED BRASS|-|3|-|4 +Brand#55|-|STANDARD POLISHED BRASS|-|49|-|4 +Brand#55|-|STANDARD POLISHED COPPER|-|19|-|4 +Brand#55|-|STANDARD POLISHED COPPER|-|36|-|4 +Brand#55|-|STANDARD POLISHED NICKEL|-|14|-|4 +Brand#55|-|STANDARD POLISHED STEEL|-|9|-|4 +Brand#55|-|STANDARD POLISHED STEEL|-|36|-|4 +Brand#12|-|LARGE BURNISHED NICKEL|-|14|-|3 +Brand#12|-|PROMO POLISHED TIN|-|3|-|3 +Brand#21|-|MEDIUM ANODIZED TIN|-|9|-|3 +Brand#22|-|PROMO BRUSHED BRASS|-|19|-|3 +Brand#22|-|PROMO BURNISHED COPPER|-|14|-|3 +Brand#43|-|STANDARD BRUSHED BRASS|-|23|-|3 +Brand#44|-|MEDIUM ANODIZED NICKEL|-|9|-|3 +Brand#53|-|MEDIUM BURNISHED BRASS|-|49|-|3 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out new file mode 100644 index 000000000000..0f4a8f8f2c42 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q17.out @@ -0,0 +1,2 @@ +1 +23512.75285714285 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out new file mode 100644 index 000000000000..9611834608d1 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q18.out @@ -0,0 +1,6 @@ +5 +Customer#000001639|-|1639|-|502886|-|1994-04-12|-|456423.88|-|312.0 +Customer#000006655|-|6655|-|29158|-|1995-10-21|-|452805.02|-|305.0 +Customer#000014110|-|14110|-|565574|-|1995-09-24|-|425099.85|-|301.0 +Customer#000001775|-|1775|-|6882|-|1997-04-09|-|408368.1|-|303.0 +Customer#000011459|-|11459|-|551136|-|1993-05-19|-|386812.74|-|308.0 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out new file mode 100644 index 000000000000..9945c5826c04 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q19.out @@ -0,0 +1,2 @@ +1 +168597.286 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out new file mode 100644 index 000000000000..98880fb6599b --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q20.out @@ -0,0 +1,10 @@ +9 +Supplier#000000157|-|,mEGorBfVIm +Supplier#000000197|-|YC2Acon6kjY3zj3Fbxs2k4Vdf7X0cd2F +Supplier#000000287|-|7a9SP7qW5Yku5PvSg +Supplier#000000378|-|FfbhyCxWvcPrO8ltp9 +Supplier#000000530|-|0qwCMwobKY OcmLyfRXlagA8ukENJv, +Supplier#000000555|-|TfB,a5bfl3Ah 3Z 74GqnNs6zKVGM +Supplier#000000557|-|jj0wUYh9K3fG5Jhdhrkuy ,4 +Supplier#000000729|-|pqck2ppy758TQpZCUAjPvlU55K3QjfL7Bi +Supplier#000000935|-|ij98czM 2KzWe7dDTOxB8sq0UfCdvrX diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out new file mode 100644 index 000000000000..c4342764a7c0 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q21.out @@ -0,0 +1,48 @@ +47 +Supplier#000000445|-|16 +Supplier#000000825|-|16 +Supplier#000000709|-|15 +Supplier#000000762|-|15 +Supplier#000000357|-|14 +Supplier#000000399|-|14 +Supplier#000000496|-|14 +Supplier#000000977|-|13 +Supplier#000000144|-|12 +Supplier#000000188|-|12 +Supplier#000000415|-|12 +Supplier#000000472|-|12 +Supplier#000000633|-|12 +Supplier#000000708|-|12 +Supplier#000000889|-|12 +Supplier#000000380|-|11 +Supplier#000000602|-|11 +Supplier#000000659|-|11 +Supplier#000000821|-|11 +Supplier#000000929|-|11 +Supplier#000000262|-|10 +Supplier#000000460|-|10 +Supplier#000000486|-|10 +Supplier#000000669|-|10 +Supplier#000000718|-|10 +Supplier#000000778|-|10 +Supplier#000000167|-|9 +Supplier#000000578|-|9 +Supplier#000000673|-|9 +Supplier#000000687|-|9 +Supplier#000000074|-|8 +Supplier#000000565|-|8 +Supplier#000000648|-|8 +Supplier#000000918|-|8 +Supplier#000000427|-|7 +Supplier#000000503|-|7 +Supplier#000000610|-|7 +Supplier#000000670|-|7 +Supplier#000000811|-|7 +Supplier#000000114|-|6 +Supplier#000000379|-|6 +Supplier#000000436|-|6 +Supplier#000000500|-|6 +Supplier#000000660|-|6 +Supplier#000000788|-|6 +Supplier#000000846|-|6 +Supplier#000000920|-|4 diff --git a/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out b/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out new file mode 100644 index 000000000000..71db28424e23 --- /dev/null +++ b/backends-clickhouse/src/test/resources/bucket-queries-output/q22.out @@ -0,0 +1,8 @@ +7 +13|-|94|-|714035.0499999999 +17|-|96|-|722560.15 +18|-|99|-|738012.5200000004 +23|-|93|-|708285.25 +29|-|85|-|632693.46 +30|-|87|-|646748.0199999998 +31|-|87|-|647372.5000000001 diff --git a/backends-clickhouse/src/test/resources/csv-data/escape_quote.csv b/backends-clickhouse/src/test/resources/csv-data/escape_quote.csv index fae8d6c41e10..328cfd341049 100644 --- a/backends-clickhouse/src/test/resources/csv-data/escape_quote.csv +++ b/backends-clickhouse/src/test/resources/csv-data/escape_quote.csv @@ -2,3 +2,11 @@ '123\'456','123\,123' '123\'456','123,123' '123\'456','123123' +'1123456789132456\\','123456789123' +'2123456789132456\n','123456789123' +'3123456789132456\\','123456789123' +'4123456789132456\\\'','123456789123' +'5123456789132456\\\\','123456789123' +'\\\\','123456789123' +'\\\1','123456789123' +'{'aa':'aa', 'b':123}','12000' diff --git a/backends-clickhouse/src/test/resources/csv-data/int_special.csv b/backends-clickhouse/src/test/resources/csv-data/int_special.csv new file mode 100644 index 000000000000..f1b8368452c5 --- /dev/null +++ b/backends-clickhouse/src/test/resources/csv-data/int_special.csv @@ -0,0 +1 @@ +15.01.02.01.002,CS-001,85.66%,15.01 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/csv-data/null_string.csv b/backends-clickhouse/src/test/resources/csv-data/null_string.csv index 8f31796a25d9..0d56a665dea5 100644 --- a/backends-clickhouse/src/test/resources/csv-data/null_string.csv +++ b/backends-clickhouse/src/test/resources/csv-data/null_string.csv @@ -1,3 +1,4 @@ -\N,1 -,2 -1,3 \ No newline at end of file +\N,1, +,2,2 +1,,\N +,, \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/text-data/cr_end_of_line/csv_with_cr_end_of_line.csv b/backends-clickhouse/src/test/resources/text-data/cr_end_of_line/csv_with_cr_end_of_line.csv new file mode 100644 index 000000000000..077ca2c84c53 --- /dev/null +++ b/backends-clickhouse/src/test/resources/text-data/cr_end_of_line/csv_with_cr_end_of_line.csv @@ -0,0 +1,2 @@ +A,110,208819249 +B,112,208819248 C,123,783434434 diff --git a/backends-clickhouse/src/test/resources/text-data/field_whitespaces/data.txt b/backends-clickhouse/src/test/resources/text-data/field_whitespaces/data.txt new file mode 100644 index 000000000000..abfed892abad --- /dev/null +++ b/backends-clickhouse/src/test/resources/text-data/field_whitespaces/data.txt @@ -0,0 +1,2 @@ +abc, cc ,ab +a,b,c diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..c24cc7adbbe9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..214269080898 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..40ab85b35d5b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +3682 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..9315c89fde56 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..f93421cb80c1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..9b9567e44cde Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..be6ce5eae1f4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3682},{"kind":"Default","name":"c_nationkey","num_defaults":100,"num_rows":3682},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3682}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..fab53631b09c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt new file mode 100644 index 000000000000..214269080898 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt new file mode 100644 index 000000000000..7f6931a12e81 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +3698 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin new file mode 100644 index 000000000000..959aea103510 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..2d08173f6da0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..32fdc6a6f39d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json new file mode 100644 index 000000000000..43e0a379e066 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3698},{"kind":"Default","name":"c_nationkey","num_defaults":160,"num_rows":3698},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3698}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..3be296ce3788 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt new file mode 100644 index 000000000000..214269080898 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt new file mode 100644 index 000000000000..3b93cf65b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +3818 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin new file mode 100644 index 000000000000..f97d3c269c90 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..b1139c1af625 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..b5fbc294b66a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json new file mode 100644 index 000000000000..a1e3accc5dd3 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3818},{"kind":"Default","name":"c_nationkey","num_defaults":110,"num_rows":3818},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3818}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..2255a1064afe Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt new file mode 100644 index 000000000000..214269080898 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/columns.txt @@ -0,0 +1,10 @@ +columns format version: 1 +8 columns: +`c_custkey` Int64 +`c_name` String +`c_address` String +`c_nationkey` Int64 +`c_phone` String +`c_acctbal` Float64 +`c_mktsegment` String +`c_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt new file mode 100644 index 000000000000..ca26457e9e82 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +3802 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin new file mode 100644 index 000000000000..07f2a8b2bf9a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..719484322a5c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..0beb0a888547 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json new file mode 100644 index 000000000000..793486590fbe --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/customer/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"c_acctbal","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_address","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_comment","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_custkey","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_mktsegment","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_name","num_defaults":0,"num_rows":3802},{"kind":"Default","name":"c_nationkey","num_defaults":140,"num_rows":3802},{"kind":"Default","name":"c_phone","num_defaults":0,"num_rows":3802}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..e5419574780a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..598d676660f8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..46989323742c --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +150183 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin new file mode 100644 index 000000000000..f8d3e600c5b1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 new file mode 100644 index 000000000000..41304769fc26 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin new file mode 100644 index 000000000000..649a72b078ef Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 new file mode 100644 index 000000000000..63002694dcaf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin new file mode 100644 index 000000000000..5c5665e66302 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 new file mode 100644 index 000000000000..fac91d4bc95d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin new file mode 100644 index 000000000000..cadd76992263 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 new file mode 100644 index 000000000000..3a41b31a55fb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin new file mode 100644 index 000000000000..2fd4cc9e30bb Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 new file mode 100644 index 000000000000..cadff0655a59 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin new file mode 100644 index 000000000000..9e14e2101498 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 new file mode 100644 index 000000000000..7d8576becc94 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin new file mode 100644 index 000000000000..e98174a2110d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 new file mode 100644 index 000000000000..e2125dbac084 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin new file mode 100644 index 000000000000..e843bafaa032 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 new file mode 100644 index 000000000000..50d265966bd9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin new file mode 100644 index 000000000000..2454f1890eed Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 new file mode 100644 index 000000000000..1d6c26cf63e9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin new file mode 100644 index 000000000000..c70b1086226e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 new file mode 100644 index 000000000000..04831b24563d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin new file mode 100644 index 000000000000..5e31a53b6c6b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 new file mode 100644 index 000000000000..e53049390cdd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin new file mode 100644 index 000000000000..c68ba1fcf3f1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 new file mode 100644 index 000000000000..34ce9f84741c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin new file mode 100644 index 000000000000..8755eeace690 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 new file mode 100644 index 000000000000..e1bd489948d7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin new file mode 100644 index 000000000000..2331c3b41100 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 new file mode 100644 index 000000000000..31994007f901 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin new file mode 100644 index 000000000000..65b450e124fe Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 new file mode 100644 index 000000000000..9316b287e25e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin new file mode 100644 index 000000000000..c1e26f0ad31b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 new file mode 100644 index 000000000000..40e53e166e62 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..d76cc0286dfa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..51d964934212 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_discount","num_defaults":13540,"num_rows":150183},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150183},{"kind":"Default","name":"l_tax","num_defaults":16500,"num_rows":150183}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..fe6bdd4eb2b4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt new file mode 100644 index 000000000000..598d676660f8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt new file mode 100644 index 000000000000..e33b55453bec --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +150025 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin new file mode 100644 index 000000000000..9466f115e62f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 new file mode 100644 index 000000000000..ffc7984d7360 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin new file mode 100644 index 000000000000..9168d376ff1d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 new file mode 100644 index 000000000000..bf4449a673b4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin new file mode 100644 index 000000000000..62db4b48f935 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 new file mode 100644 index 000000000000..98d67588623f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin new file mode 100644 index 000000000000..5f601760effa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 new file mode 100644 index 000000000000..370a22b155b0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin new file mode 100644 index 000000000000..436a54049310 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 new file mode 100644 index 000000000000..e96b24a0d6ca Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin new file mode 100644 index 000000000000..e3d926af2a9f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 new file mode 100644 index 000000000000..4a6b1f63a8ce Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin new file mode 100644 index 000000000000..01c90d9fe3e3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 new file mode 100644 index 000000000000..f40a82e3481e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin new file mode 100644 index 000000000000..7fb3c626269a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 new file mode 100644 index 000000000000..ec0641f72db9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin new file mode 100644 index 000000000000..e29f5881ef5a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 new file mode 100644 index 000000000000..741b0f65fda0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin new file mode 100644 index 000000000000..17eb397a0593 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 new file mode 100644 index 000000000000..7ab1d7e75a0a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin new file mode 100644 index 000000000000..ff83da4c983f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 new file mode 100644 index 000000000000..aeca561e30c4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin new file mode 100644 index 000000000000..f4d821d808f2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 new file mode 100644 index 000000000000..af5d5b54f42b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin new file mode 100644 index 000000000000..448c814df204 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 new file mode 100644 index 000000000000..fd2843cf2faa Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin new file mode 100644 index 000000000000..5d4b26a7bc82 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 new file mode 100644 index 000000000000..f60f9da6f592 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin new file mode 100644 index 000000000000..6532245dc1c9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 new file mode 100644 index 000000000000..c494259dfe42 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin new file mode 100644 index 000000000000..6e835238a545 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 new file mode 100644 index 000000000000..d7a2cf92aac6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..f577026504b2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json new file mode 100644 index 000000000000..4b8c99383d9f --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_discount","num_defaults":13620,"num_rows":150025},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150025},{"kind":"Default","name":"l_tax","num_defaults":16260,"num_rows":150025}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..3887c120e65d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt new file mode 100644 index 000000000000..598d676660f8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt new file mode 100644 index 000000000000..45c374d3b635 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +150103 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin new file mode 100644 index 000000000000..408e74b95e9a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 new file mode 100644 index 000000000000..421d1eaf1e69 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin new file mode 100644 index 000000000000..2fbd41949491 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 new file mode 100644 index 000000000000..8829d7381ed7 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin new file mode 100644 index 000000000000..75f3fbc03cac Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 new file mode 100644 index 000000000000..e37555a4008e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin new file mode 100644 index 000000000000..2780d5f256c0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 new file mode 100644 index 000000000000..03af8943f3cf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin new file mode 100644 index 000000000000..059048d52bba Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 new file mode 100644 index 000000000000..fe3742e5d9ca Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin new file mode 100644 index 000000000000..f79b740e1425 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 new file mode 100644 index 000000000000..98281152d342 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin new file mode 100644 index 000000000000..db4603c92bc9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 new file mode 100644 index 000000000000..175401d84327 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin new file mode 100644 index 000000000000..63ab678d94fd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 new file mode 100644 index 000000000000..703cd3a9106a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin new file mode 100644 index 000000000000..b1a8a224d92e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 new file mode 100644 index 000000000000..055ca8f0a9ba Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin new file mode 100644 index 000000000000..b46766a56b65 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 new file mode 100644 index 000000000000..4a065f4bd8a2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin new file mode 100644 index 000000000000..6db9a44fac28 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 new file mode 100644 index 000000000000..0c1af0fc822b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin new file mode 100644 index 000000000000..edf48da70d82 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 new file mode 100644 index 000000000000..9a625ad4ab2e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin new file mode 100644 index 000000000000..65cac6d9aaa2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 new file mode 100644 index 000000000000..2de907621bb9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin new file mode 100644 index 000000000000..85d79cd828a5 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 new file mode 100644 index 000000000000..dbd1a9fa0775 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin new file mode 100644 index 000000000000..3bd908ef18b8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 new file mode 100644 index 000000000000..9a9ead3f9362 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin new file mode 100644 index 000000000000..5ae494b93b72 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 new file mode 100644 index 000000000000..024467b188d9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..45ac79b66435 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json new file mode 100644 index 000000000000..6dcc4458e81d --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_discount","num_defaults":13330,"num_rows":150103},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150103},{"kind":"Default","name":"l_tax","num_defaults":16800,"num_rows":150103}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..c3eafce89b34 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt new file mode 100644 index 000000000000..598d676660f8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/columns.txt @@ -0,0 +1,18 @@ +columns format version: 1 +16 columns: +`l_orderkey` Int64 +`l_partkey` Int64 +`l_suppkey` Int64 +`l_linenumber` Int64 +`l_quantity` Float64 +`l_extendedprice` Float64 +`l_discount` Float64 +`l_tax` Float64 +`l_returnflag` String +`l_linestatus` String +`l_shipdate` Date +`l_commitdate` Date +`l_receiptdate` Date +`l_shipinstruct` String +`l_shipmode` String +`l_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt new file mode 100644 index 000000000000..86f6ce8ede19 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +150261 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin new file mode 100644 index 000000000000..95f69fd6418e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 new file mode 100644 index 000000000000..5f4cc6326a91 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_comment.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin new file mode 100644 index 000000000000..1b1f8fbed133 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 new file mode 100644 index 000000000000..a054b7cc4185 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_commitdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin new file mode 100644 index 000000000000..485983463b67 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 new file mode 100644 index 000000000000..dd85512cc40a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_discount.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin new file mode 100644 index 000000000000..a9b92b2d8112 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 new file mode 100644 index 000000000000..7ef7b0963d2d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_extendedprice.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin new file mode 100644 index 000000000000..1f5763119603 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 new file mode 100644 index 000000000000..bc8886112c84 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linenumber.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin new file mode 100644 index 000000000000..adb867de7c8f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 new file mode 100644 index 000000000000..6b21a1ca465a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_linestatus.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin new file mode 100644 index 000000000000..f4840d2ca245 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 new file mode 100644 index 000000000000..0067fc50da8c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_orderkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin new file mode 100644 index 000000000000..e82138df2fb4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 new file mode 100644 index 000000000000..1819a433430d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_partkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin new file mode 100644 index 000000000000..21a469330612 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 new file mode 100644 index 000000000000..afed0101b3c4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_quantity.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin new file mode 100644 index 000000000000..7a6c267cb9cc Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 new file mode 100644 index 000000000000..e4adf9e2dd70 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_receiptdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin new file mode 100644 index 000000000000..b0d48b41c5ec Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 new file mode 100644 index 000000000000..29bd96ae152a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_returnflag.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin new file mode 100644 index 000000000000..dd8141efe33c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 new file mode 100644 index 000000000000..4185e02b69b2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipdate.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin new file mode 100644 index 000000000000..53838fb3559f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 new file mode 100644 index 000000000000..108307a66b6e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipinstruct.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin new file mode 100644 index 000000000000..bc3dd7a22ad0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 new file mode 100644 index 000000000000..d906135ee587 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_shipmode.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin new file mode 100644 index 000000000000..3df367f9e740 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 new file mode 100644 index 000000000000..4144cdf14a38 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_suppkey.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin new file mode 100644 index 000000000000..95d648c9397c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 new file mode 100644 index 000000000000..47a6e77fc576 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/l_tax.cmrk2 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..32ae4d72f47a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json new file mode 100644 index 000000000000..91fd12eedfb2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/lineitem/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"l_comment","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_commitdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_discount","num_defaults":13460,"num_rows":150261},{"kind":"Default","name":"l_extendedprice","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_linenumber","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_linestatus","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_orderkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_partkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_quantity","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_receiptdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_returnflag","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipdate","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipinstruct","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_shipmode","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_suppkey","num_defaults":0,"num_rows":150261},{"kind":"Default","name":"l_tax","num_defaults":16870,"num_rows":150261}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..dbdb1d37e197 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..1b0cd5e295ec --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/columns.txt @@ -0,0 +1,6 @@ +columns format version: 1 +4 columns: +`n_nationkey` Int64 +`n_name` String +`n_regionkey` Int64 +`n_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..410b14d2ce6f --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..ffa999a7f9d2 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..a3ed1f653997 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..f323dcd7db25 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..6caa0c7892a8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/nation/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"n_comment","num_defaults":0,"num_rows":25},{"kind":"Default","name":"n_name","num_defaults":0,"num_rows":25},{"kind":"Default","name":"n_nationkey","num_defaults":1,"num_rows":25},{"kind":"Default","name":"n_regionkey","num_defaults":5,"num_rows":25}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..85ffbd61c76b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..6dfe31e78fef --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..ae1d2dbe5472 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +37359 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..e63f8b3e5a9b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..af4873bbbdc3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..abf42eed670a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..11c8f61315bb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37359},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37359},{"kind":"Sparse","name":"o_shippriority","num_defaults":37359,"num_rows":37359},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37359}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..ba6219414354 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt new file mode 100644 index 000000000000..6dfe31e78fef --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt new file mode 100644 index 000000000000..7b8330dab9c4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +37652 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin new file mode 100644 index 000000000000..2f3e820560a0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..71770cd09f35 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..76052c3f6b33 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json new file mode 100644 index 000000000000..3c3a6d6c6988 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37652},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37652},{"kind":"Sparse","name":"o_shippriority","num_defaults":37652,"num_rows":37652},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37652}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..cfd8888b37df Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt new file mode 100644 index 000000000000..6dfe31e78fef --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt new file mode 100644 index 000000000000..c585e5375cc4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +37641 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin new file mode 100644 index 000000000000..64346528d5f4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..014387b46a38 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..8983040b4f74 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json new file mode 100644 index 000000000000..155cd493ede5 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37641},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37641},{"kind":"Sparse","name":"o_shippriority","num_defaults":37641,"num_rows":37641},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37641}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..8319c3251521 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt new file mode 100644 index 000000000000..6dfe31e78fef --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`o_orderkey` Int64 +`o_custkey` Int64 +`o_orderstatus` String +`o_totalprice` Float64 +`o_orderdate` Date +`o_orderpriority` String +`o_clerk` String +`o_shippriority` Int64 +`o_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt new file mode 100644 index 000000000000..fec61cad7645 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +37348 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin new file mode 100644 index 000000000000..e72837d98dd1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..5a885f6cefcf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..743b7c6dbd60 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json new file mode 100644 index 000000000000..239572db2470 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/orders/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"o_clerk","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_comment","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_custkey","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderdate","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderkey","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderpriority","num_defaults":0,"num_rows":37348},{"kind":"Default","name":"o_orderstatus","num_defaults":0,"num_rows":37348},{"kind":"Sparse","name":"o_shippriority","num_defaults":37348,"num_rows":37348},{"kind":"Default","name":"o_totalprice","num_defaults":0,"num_rows":37348}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..d75cf1d42a8f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..03af5b613ab8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..9f945bb2547e --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +4956 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..eee5662f5e10 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..63e3f1e3eaa9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..1111223bf410 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..2dba9a4fb063 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":4956},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":4956}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..5776a23ba4dd Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt new file mode 100644 index 000000000000..03af5b613ab8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt new file mode 100644 index 000000000000..c65ed0e61653 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +4917 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin new file mode 100644 index 000000000000..4db23bebe40b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..8adb51f83063 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..9d0261e14c03 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json new file mode 100644 index 000000000000..56da900c9fa4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":4917},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":4917}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..80db7774f31d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt new file mode 100644 index 000000000000..03af5b613ab8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt new file mode 100644 index 000000000000..edbcad025b23 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +5044 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin new file mode 100644 index 000000000000..7981580a9e02 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..a157da55307d Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..28a1b5f36710 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json new file mode 100644 index 000000000000..a6e9dc7b61b4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":5044},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":5044}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..3fad79d20e97 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt new file mode 100644 index 000000000000..03af5b613ab8 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/columns.txt @@ -0,0 +1,11 @@ +columns format version: 1 +9 columns: +`p_partkey` Int64 +`p_name` String +`p_mfgr` String +`p_brand` String +`p_type` String +`p_size` Int64 +`p_container` String +`p_retailprice` Float64 +`p_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt new file mode 100644 index 000000000000..0caca7bf9e15 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +5083 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin new file mode 100644 index 000000000000..c33d22e5b517 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..114907af6932 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..4cf8a8543b72 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json new file mode 100644 index 000000000000..98a4a89817b4 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/part/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"p_brand","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_comment","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_container","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_mfgr","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_name","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_partkey","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_retailprice","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_size","num_defaults":0,"num_rows":5083},{"kind":"Default","name":"p_type","num_defaults":0,"num_rows":5083}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..925f0214c1f9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..b238d80aa532 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..e76cc2d99464 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +19692 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..4596e07e8024 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..7d205cc0ba92 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..1a6aa1025d1a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..e9f2684142a3 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":19692},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":19692}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..7c4fbc1075e9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt new file mode 100644 index 000000000000..b238d80aa532 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt new file mode 100644 index 000000000000..f8ff62878b46 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/count.txt @@ -0,0 +1 @@ +19800 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin new file mode 100644 index 000000000000..64ba656fe0ea Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..125b815c52b4 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..97e1cdc21faf Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json new file mode 100644 index 000000000000..ab84853e95b2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00000/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":19800},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":19800}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..87aeeeb3ca08 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt new file mode 100644 index 000000000000..b238d80aa532 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt new file mode 100644 index 000000000000..7657197913e2 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/count.txt @@ -0,0 +1 @@ +20308 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin new file mode 100644 index 000000000000..76fc644c0202 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..67cb661bc7e6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..d9174b671f28 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json new file mode 100644 index 000000000000..5a366eba2d5c --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":20308},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":20308}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt new file mode 100644 index 000000000000..f06dcd26da9f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt new file mode 100644 index 000000000000..b238d80aa532 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/columns.txt @@ -0,0 +1,7 @@ +columns format version: 1 +5 columns: +`ps_partkey` Int64 +`ps_suppkey` Int64 +`ps_availqty` Int64 +`ps_supplycost` Float64 +`ps_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt new file mode 100644 index 000000000000..2a32e263681f --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/count.txt @@ -0,0 +1 @@ +20200 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin new file mode 100644 index 000000000000..d1109634c1d9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 new file mode 100644 index 000000000000..5f1afe89e2c6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx new file mode 100644 index 000000000000..428d2a33ad59 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json new file mode 100644 index 000000000000..f80553ad2f37 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/partsupp/00001/all_2_2_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"ps_availqty","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_comment","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_partkey","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_suppkey","num_defaults":0,"num_rows":20200},{"kind":"Default","name":"ps_supplycost","num_defaults":0,"num_rows":20200}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..6310184e154b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..c22e2b6776e6 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/columns.txt @@ -0,0 +1,5 @@ +columns format version: 1 +3 columns: +`r_regionkey` Int64 +`r_name` String +`r_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..7813681f5b41 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..6c763f4d18ed Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..b864a4758a35 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..ba2a8d00588c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..96475e59e771 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/region/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"r_comment","num_defaults":0,"num_rows":5},{"kind":"Default","name":"r_name","num_defaults":0,"num_rows":5},{"kind":"Default","name":"r_regionkey","num_defaults":1,"num_rows":5}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt new file mode 100644 index 000000000000..18b4a3c5fdc9 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/checksums.txt differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt new file mode 100644 index 000000000000..c1b3eb58785b --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/columns.txt @@ -0,0 +1,9 @@ +columns format version: 1 +7 columns: +`s_suppkey` Int64 +`s_name` String +`s_address` String +`s_nationkey` Int64 +`s_phone` String +`s_acctbal` Float64 +`s_comment` String diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt new file mode 100644 index 000000000000..e37d32abba42 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/count.txt @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin new file mode 100644 index 000000000000..0b215ffe47c8 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.bin differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 new file mode 100644 index 000000000000..216415065588 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/data.cmrk3 differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt new file mode 100644 index 000000000000..061d1280b897 --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/default_compression_codec.txt @@ -0,0 +1 @@ +CODEC(LZ4) \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt new file mode 100644 index 000000000000..c227083464fb --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/metadata_version.txt @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx new file mode 100644 index 000000000000..2540ced74cd1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/primary.cidx differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json new file mode 100644 index 000000000000..67c8884ba4fe --- /dev/null +++ b/backends-clickhouse/src/test/resources/tpch-data-bucket/mergetree_bucket/supplier/00000/all_1_1_0/serialization.json @@ -0,0 +1 @@ +{"columns":[{"kind":"Default","name":"s_acctbal","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_address","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_comment","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_name","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_nationkey","num_defaults":36,"num_rows":1000},{"kind":"Default","name":"s_phone","num_defaults":0,"num_rows":1000},{"kind":"Default","name":"s_suppkey","num_defaults":0,"num_rows":1000}],"version":0} \ No newline at end of file diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet new file mode 100644 index 000000000000..0fa4d5944e61 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet new file mode 100644 index 000000000000..d1b02a02e241 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00000-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet new file mode 100644 index 000000000000..4a2e590c2780 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet new file mode 100644 index 000000000000..ad43a2333d8e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/customer/part-00001-72fdd3c7-0c9d-453e-abca-c9caa7219875_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet new file mode 100644 index 000000000000..d69bc752d08e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet new file mode 100644 index 000000000000..69018b7f4570 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00000-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet new file mode 100644 index 000000000000..86084b590de3 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet new file mode 100644 index 000000000000..1a3b0f91b0b1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/lineitem/part-00001-8b86c10d-22d3-4800-8d49-0de982c95d93_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet new file mode 100644 index 000000000000..c10d7bb9b151 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/nation/part-00000-008a5ae9-3c9c-42f1-bef3-81cc60b3f4ab_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet new file mode 100644 index 000000000000..938cc20de48c Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet new file mode 100644 index 000000000000..eb80f5f0ce36 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00000-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet new file mode 100644 index 000000000000..879e358c2d35 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet new file mode 100644 index 000000000000..eef0227873c6 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/orders/part-00001-a527ff3d-0d73-4f1a-a1a2-78c1beaf7ece_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet new file mode 100644 index 000000000000..2fded2af5a0e Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet new file mode 100644 index 000000000000..3d342e80a78f Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00000-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet new file mode 100644 index 000000000000..3180a2724bb1 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet new file mode 100644 index 000000000000..37cc223cd652 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/part/part-00001-0c39a257-1f7a-4c6e-85a2-7d21fefbef78_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet new file mode 100644 index 000000000000..c25e5565222b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet new file mode 100644 index 000000000000..40735373a057 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00000-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet new file mode 100644 index 000000000000..e59dc1e8844a Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet new file mode 100644 index 000000000000..e53d6517ce6b Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/partsupp/part-00001-a989f762-9541-4d30-a8cf-96d8e695e188_00001.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet new file mode 100644 index 000000000000..74955d0483e0 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/region/part-00000-4f02eea9-c038-4c8b-ab09-5f36588cb8be_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet new file mode 100644 index 000000000000..c9b45ea12559 Binary files /dev/null and b/backends-clickhouse/src/test/resources/tpch-data-bucket/parquet_bucket/supplier/part-00000-2d14f2f1-565e-49d3-8f00-cf5d3a8466a3_00000.c000.snappy.parquet differ diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseColumnarShuffleAQESuite.scala index d9472c5188c3..c54124e0644f 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseColumnarShuffleAQESuite.scala @@ -18,7 +18,7 @@ package io.glutenproject.execution import org.apache.spark.SparkConf import org.apache.spark.sql.execution.CoalescedPartitionSpec -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ColumnarAQEShuffleReadExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec} class GlutenClickHouseColumnarShuffleAQESuite extends GlutenClickHouseTPCHAbstractSuite @@ -45,7 +45,7 @@ class GlutenClickHouseColumnarShuffleAQESuite assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) val colCustomShuffleReaderExecs = collect(df.queryExecution.executedPlan) { - case csr: ColumnarAQEShuffleReadExec => csr + case csr: AQEShuffleReadExec => csr } assert(colCustomShuffleReaderExecs.size == 2) val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseDecimalSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseDecimalSuite.scala index 698ffaeaacc3..d7b4a0c57d48 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseDecimalSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseDecimalSuite.scala @@ -19,9 +19,12 @@ package io.glutenproject.execution import org.apache.spark.SparkConf import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.{col, rand, when} import org.apache.spark.sql.types._ +import java.io.File import java.util + case class DataTypesWithNonPrimitiveType( string_field: String, int_field: java.lang.Integer, @@ -40,11 +43,17 @@ class GlutenClickHouseDecimalSuite rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" override protected val queriesResults: String = rootPath + "queries-output" - override protected def createTPCHNullableTables(): Unit = {} - override protected def createTPCHNotNullTables(): Unit = {} - override protected def sparkConf: SparkConf = super.sparkConf + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "sort") + .set("spark.io.compression.codec", "snappy") + .set("spark.sql.shuffle.partitions", "5") + .set("spark.sql.autoBroadcastJoinThreshold", "10MB") + .set("spark.gluten.sql.columnar.backend.ch.use.v2", "false") + .set("spark.sql.decimalOperations.allowPrecisionLoss", "false") + } override def beforeAll(): Unit = { super.beforeAll() @@ -54,6 +63,246 @@ class GlutenClickHouseDecimalSuite } private val decimalTable: String = "decimal_table" + private val decimalTPCHTables: Seq[DecimalType] = Seq.apply(DecimalType.apply(18, 8)) + + override protected val createNullableTables = true + + override protected def createTPCHNullableTables(): Unit = { + decimalTPCHTables.foreach(createDecimalTables) + } + + private def createDecimalTables(dataType: DecimalType): Unit = { + spark.sql(s"DROP database IF EXISTS decimal_${dataType.precision}_${dataType.scale}") + spark.sql(s"create database IF not EXISTS decimal_${dataType.precision}_${dataType.scale}") + spark.sql(s"use decimal_${dataType.precision}_${dataType.scale}") + + // first process the parquet data to: + // 1. make every column nullable in schema (optional rather than required) + // 2. salt some null values randomly + val saltedTablesPath = tablesPath + s"-decimal_${dataType.precision}_${dataType.scale}" + withSQLConf(vanillaSparkConfs(): _*) { + Seq("customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier") + .map( + tableName => { + val originTablePath = tablesPath + "/" + tableName + spark.read.parquet(originTablePath).createTempView(tableName + "_ori") + + val sql = tableName match { + case "customer" => + s""" + |select + | c_custkey,c_name,c_address,c_nationkey,c_phone, + | cast(c_acctbal as decimal(${dataType.precision},${dataType.scale})), + | c_mktsegment,c_comment + |from ${tableName}_ori""".stripMargin + case "lineitem" => + s""" + |select + | l_orderkey,l_partkey,l_suppkey,l_linenumber, + | cast(l_quantity as decimal(${dataType.precision},${dataType.scale})), + | cast(l_extendedprice as decimal(${dataType.precision},${dataType.scale})), + | cast(l_discount as decimal(${dataType.precision},${dataType.scale})), + | cast(l_tax as decimal(${dataType.precision},${dataType.scale})), + | l_returnflag,l_linestatus,l_shipdate,l_commitdate,l_receiptdate, + | l_shipinstruct,l_shipmode,l_comment + |from ${tableName}_ori """.stripMargin + case "orders" => + s""" + |select + | o_orderkey,o_custkey,o_orderstatus, + | cast(o_totalprice as decimal(${dataType.precision},${dataType.scale})), + | o_orderdate, + | o_orderpriority,o_clerk,o_shippriority,o_comment + |from ${tableName}_ori + |""".stripMargin + case "part" => + s""" + |select + | p_partkey,p_name,p_mfgr,p_brand,p_type,p_size,p_container, + | cast(p_retailprice as decimal(${dataType.precision},${dataType.scale})), + | p_comment + |from ${tableName}_ori + |""".stripMargin + case "partsupp" => + s""" + |select + | ps_partkey,ps_suppkey,ps_availqty, + | cast(ps_supplycost as decimal(${dataType.precision},${dataType.scale})), + | ps_comment + |from ${tableName}_ori + |""".stripMargin + case "supplier" => + s""" + |select + | s_suppkey,s_name,s_address,s_nationkey,s_phone, + | cast(s_acctbal as decimal(${dataType.precision},${dataType.scale})),s_comment + |from ${tableName}_ori + |""".stripMargin + case _ => s"select * from ${tableName}_ori" + } + + val df = spark.sql(sql).toDF() + var salted_df: Option[DataFrame] = None + for (c <- df.schema) { + salted_df = Some((salted_df match { + case Some(x) => x + case None => df + }).withColumn(c.name, when(rand() < 0.1, null).otherwise(col(c.name)))) + } + + val currentSaltedTablePath = saltedTablesPath + "/" + tableName + val file = new File(currentSaltedTablePath) + if (file.exists()) { + file.delete() + } + + salted_df.get.write.parquet(currentSaltedTablePath) + }) + } + + val customerData = saltedTablesPath + "/customer" + spark.sql(s"DROP TABLE IF EXISTS customer") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal decimal(${dataType.precision},${dataType.scale}), + | c_mktsegment string, + | c_comment string) + | USING PARQUET LOCATION '$customerData' + |""".stripMargin) + + val lineitemData = saltedTablesPath + "/lineitem" + spark.sql(s"DROP TABLE IF EXISTS lineitem") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS lineitem ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity decimal(${dataType.precision},${dataType.scale}), + | l_extendedprice decimal(${dataType.precision},${dataType.scale}), + | l_discount decimal(${dataType.precision},${dataType.scale}), + | l_tax decimal(${dataType.precision},${dataType.scale}), + | 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 PARQUET LOCATION '$lineitemData' + |""".stripMargin) + + val nationData = saltedTablesPath + "/nation" + spark.sql(s"DROP TABLE IF EXISTS nation") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING PARQUET LOCATION '$nationData' + |""".stripMargin) + + val regionData = saltedTablesPath + "/region" + spark.sql(s"DROP TABLE IF EXISTS region") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING PARQUET LOCATION '$regionData' + |""".stripMargin) + + val ordersData = saltedTablesPath + "/orders" + spark.sql(s"DROP TABLE IF EXISTS orders") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice decimal(${dataType.precision},${dataType.scale}), + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING PARQUET LOCATION '$ordersData' + |""".stripMargin) + + val partData = saltedTablesPath + "/part" + spark.sql(s"DROP TABLE IF EXISTS part") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice decimal(${dataType.precision},${dataType.scale}), + | p_comment string) + | USING PARQUET LOCATION '$partData' + |""".stripMargin) + + val partsuppData = saltedTablesPath + "/partsupp" + spark.sql(s"DROP TABLE IF EXISTS partsupp") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost decimal(${dataType.precision},${dataType.scale}), + | ps_comment string) + | USING PARQUET LOCATION '$partsuppData' + |""".stripMargin) + + val supplierData = saltedTablesPath + "/supplier" + spark.sql(s"DROP TABLE IF EXISTS supplier") + spark.sql(s""" + | CREATE TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal decimal(${dataType.precision},${dataType.scale}), + | s_comment string) + | USING PARQUET LOCATION '$supplierData' + |""".stripMargin) + + val result = spark + .sql(s""" + | show tables; + |""".stripMargin) + .collect() + assert(result.size == 16) + spark.sql(s"use default") + } + + override protected def runTPCHQuery( + queryNum: Int, + tpchQueries: String = tpchQueries, + queriesResults: String = queriesResults, + compareResult: Boolean = true, + noFallBack: Boolean = true)(customCheck: DataFrame => Unit): Unit = { + decimalTPCHTables.foreach( + decimalType => { + spark.sql(s"use decimal_${decimalType.precision}_${decimalType.scale}") + compareTPCHQueryAgainstVanillaSpark(queryNum, tpchQueries, customCheck, noFallBack) + spark.sql(s"use default") + }) + } + + test("TPCH Q20") { + runTPCHQuery(20)(_ => {}) + } test("fix decimal precision overflow") { val sql = diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseFileFormatSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseFileFormatSuite.scala index 2dbd89748740..40752eb9c2e5 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseFileFormatSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseFileFormatSuite.scala @@ -808,6 +808,46 @@ class GlutenClickHouseFileFormatSuite case f: FileSourceScanExecTransformer => f } assert(csvFileScan.size == 1) + + val no_quote_path = csvDataPath + "/no_quote.csv" + val no_quote_option = new util.HashMap[String, String]() + no_quote_option.put("delimiter", ",") + no_quote_option.put("header", "false") + no_quote_option.put("quote", "") + + val no_quote_schema = StructType.apply( + Seq( + StructField.apply("a", StringType, nullable = true), + StructField.apply("b", StringType, nullable = true) + )) + + val data = new util.ArrayList[Row]() + data.add(Row("\'abc\'de\'", "\"abc\"de\"")) + + spark + .createDataFrame(data, schema) + .write + .mode("overwrite") + .format("csv") + .options(no_quote_option) + .save(no_quote_path) + + spark.read + .options(no_quote_option) + .schema(no_quote_schema) + .csv(no_quote_path) + .toDF() + .createTempView("no_quote_table") + + withSQLConf(( + "spark.gluten.sql.columnar.backend.ch.runtime_settings.use_excel_serialization.quote_strict", + "true" + )) { + compareResultsAgainstVanillaSpark( + "select * from no_quote_table", + compareResult = true, + _ => {}) + } } test("test read excel with header") { @@ -1108,32 +1148,77 @@ class GlutenClickHouseFileFormatSuite } test("issue-2881 null string test") { - val file_path = csvDataPath + "/null_string.csv" - val schema = StructType.apply( - Seq( - StructField.apply("c1", StringType, nullable = true), - StructField.apply("c2", ShortType, nullable = true) - )) + withSQLConf( + ( + "spark.gluten.sql.columnar.backend.ch.runtime_settings." + + "use_excel_serialization.empty_as_null", + "true")) { + val file_path = csvDataPath + "/null_string.csv" + val schema = StructType.apply( + Seq( + StructField.apply("c1", StringType, nullable = true), + StructField.apply("c2", ShortType, nullable = true), + StructField.apply("c3", StringType, nullable = true) + )) + + val options = new util.HashMap[String, String]() + options.put("delimiter", ",") + + val df = spark.read + .options(options) + .schema(schema) + .csv(file_path) + .toDF() - val options = new util.HashMap[String, String]() - options.put("delimiter", ",") + val dataCorrect = new util.ArrayList[Row]() + dataCorrect.add(Row(null, 1.toShort, null)) + dataCorrect.add(Row(null, 2.toShort, "2")) + dataCorrect.add(Row("1", null, null)) + dataCorrect.add(Row(null, null, null)) - val df = spark.read - .options(options) - .schema(schema) - .csv(file_path) - .toDF() + var expectedAnswer: Seq[Row] = null + withSQLConf(vanillaSparkConfs(): _*) { + expectedAnswer = spark.createDataFrame(dataCorrect, schema).toDF().collect() + } + checkAnswer(df, expectedAnswer) + } + } - val dataCorrect = new util.ArrayList[Row]() - dataCorrect.add(Row(null, 1.toShort)) - dataCorrect.add(Row(null, 2.toShort)) - dataCorrect.add(Row("1", 3.toShort)) + test("issue-3542 null string test") { + withSQLConf( + ( + "spark.gluten.sql.columnar.backend.ch.runtime_settings." + + "use_excel_serialization.empty_as_null", + "false")) { + val file_path = csvDataPath + "/null_string.csv" + val schema = StructType.apply( + Seq( + StructField.apply("c1", StringType, nullable = true), + StructField.apply("c2", ShortType, nullable = true), + StructField.apply("c3", StringType, nullable = true) + )) + + val options = new util.HashMap[String, String]() + options.put("delimiter", ",") + + val df = spark.read + .options(options) + .schema(schema) + .csv(file_path) + .toDF() - var expectedAnswer: Seq[Row] = null - withSQLConf(vanillaSparkConfs(): _*) { - expectedAnswer = spark.createDataFrame(dataCorrect, schema).toDF().collect() + val dataCorrect = new util.ArrayList[Row]() + dataCorrect.add(Row(null, 1.toShort, "")) + dataCorrect.add(Row("", 2.toShort, "2")) + dataCorrect.add(Row("1", null, null)) + dataCorrect.add(Row("", null, "")) + + var expectedAnswer: Seq[Row] = null + withSQLConf(vanillaSparkConfs(): _*) { + expectedAnswer = spark.createDataFrame(dataCorrect, schema).toDF().collect() + } + checkAnswer(df, expectedAnswer) } - checkAnswer(df, expectedAnswer) } test("test integer read with sign at the end of line") { @@ -1188,6 +1273,74 @@ class GlutenClickHouseFileFormatSuite ) } + test("issues-3609 int read test") { + withSQLConf( + ( + "spark.gluten.sql.columnar.backend.ch.runtime_settings." + + "use_excel_serialization.number_force", + "false")) { + val csv_path = csvDataPath + "/int_special.csv" + val options = new util.HashMap[String, String]() + options.put("delimiter", ",") + options.put("header", "false") + val schema = StructType.apply( + Seq( + StructField.apply("a", IntegerType, nullable = true), + StructField.apply("b", IntegerType, nullable = true), + StructField.apply("c", IntegerType, nullable = true), + StructField.apply("d", IntegerType, nullable = true) + )) + + val df = spark.read + .options(options) + .schema(schema) + .csv(csv_path) + .toDF() + + val dataCorrect = new util.ArrayList[Row]() + dataCorrect.add(Row(null, null, null, 15)) + + var expectedAnswer: Seq[Row] = null + withSQLConf(vanillaSparkConfs(): _*) { + expectedAnswer = spark.createDataFrame(dataCorrect, schema).toDF().collect() + } + checkAnswer(df, expectedAnswer) + } + + withSQLConf( + ( + "spark.gluten.sql.columnar.backend.ch.runtime_settings." + + "use_excel_serialization.number_force", + "true")) { + val csv_path = csvDataPath + "/int_special.csv" + val options = new util.HashMap[String, String]() + options.put("delimiter", ",") + options.put("header", "false") + val schema = StructType.apply( + Seq( + StructField.apply("a", IntegerType, nullable = true), + StructField.apply("b", IntegerType, nullable = true), + StructField.apply("c", IntegerType, nullable = true), + StructField.apply("d", IntegerType, nullable = true) + )) + + val df = spark.read + .options(options) + .schema(schema) + .csv(csv_path) + .toDF() + + val dataCorrect = new util.ArrayList[Row]() + dataCorrect.add(Row(15, -1, 85, 15)) + + var expectedAnswer: Seq[Row] = null + withSQLConf(vanillaSparkConfs(): _*) { + expectedAnswer = spark.createDataFrame(dataCorrect, schema).toDF().collect() + } + checkAnswer(df, expectedAnswer) + } + } + def createEmptyParquet(): String = { val data = spark.sparkContext.emptyRDD[Row] val schema = new StructType() diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseHiveTableSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseHiveTableSuite.scala index 48a1fc2af17f..0109859c7ff6 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseHiveTableSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseHiveTableSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.Path -import java.io.File +import java.io.{File, PrintWriter} import java.sql.Timestamp case class AllDataTypesWithComplextType( @@ -92,7 +92,6 @@ class GlutenClickHouseHiveTableSuite() .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceshuffledhashjoin", "true") .set("spark.gluten.sql.parquet.maxmin.index", "true") .set( "spark.sql.warehouse.dir", @@ -972,4 +971,84 @@ class GlutenClickHouseHiveTableSuite() spark.sql("DROP TABLE test_tbl_3337") } + test("test hive read recursive dirs") { + val path = new Path(sparkConf.get("spark.sql.warehouse.dir")) + val create_test_file_recursive = + "create external table if not exists test_file_recursive (" + + "int_field int" + + ") row format delimited fields terminated by ',' stored as textfile " + + "LOCATION 'file://" + path + "/test_file_recursive'" + spark.sql(create_test_file_recursive) + + val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) + val tablePath = path.toUri.getPath + "/test_file_recursive" + val recursivePath = tablePath + "/subDir1/subDir2" + val succ = fs.mkdirs(new Path(recursivePath)) + assert(succ, true) + val recursiveFile = recursivePath + "/file1.txt" + val writer = new PrintWriter(new File(recursiveFile)) + writer.write("10") + writer.close() + + val sql = + s""" + | select int_field from test_file_recursive + |""".stripMargin + + withSQLConf(("mapreduce.input.fileinputformat.input.dir.recursive", "true")) { + compareResultsAgainstVanillaSpark( + sql, + compareResult = true, + df => { + assert(df.collect().length == 1) + } + ) + } + } + + test("GLUTEN-3552: Bug fix csv field whitespaces") { + val data_path = rootPath + "/text-data/field_whitespaces" + spark.sql(s""" + | CREATE TABLE test_tbl_3552( + | a string, + | b string, + | c string) + | ROW FORMAT SERDE + | 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ( + | 'field.delim'=',' + | ) + | STORED AS INPUTFORMAT + | 'org.apache.hadoop.mapred.TextInputFormat' + |OUTPUTFORMAT + | 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + |LOCATION '$data_path' + |""".stripMargin) + val select_sql = "select * from test_tbl_3552" + compareResultsAgainstVanillaSpark(select_sql, compareResult = true, _ => {}) + spark.sql("DROP TABLE test_tbl_3552") + } + + test("GLUTEN-3548: Bug fix csv allow cr end of line") { + val data_path = rootPath + "/text-data/cr_end_of_line" + spark.sql(s""" + | CREATE TABLE test_tbl_3548( + | a string, + | b string, + | c string) + | ROW FORMAT SERDE + | 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + |WITH SERDEPROPERTIES ( + | 'field.delim'=',' + | ) + | STORED AS INPUTFORMAT + | 'org.apache.hadoop.mapred.TextInputFormat' + |OUTPUTFORMAT + | 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + |LOCATION '$data_path' + |""".stripMargin) + val select_sql = "select * from test_tbl_3548" + compareResultsAgainstVanillaSpark(select_sql, compareResult = true, _ => {}) + spark.sql("DROP TABLE test_tbl_3548") + } } diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseNativeWriteTableSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseNativeWriteTableSuite.scala index 7517c5aff85e..d3e910191101 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseNativeWriteTableSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseNativeWriteTableSuite.scala @@ -72,7 +72,6 @@ class GlutenClickHouseNativeWriteTableSuite .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceshuffledhashjoin", "true") // TODO: support default ANSI policy .set("spark.sql.storeAssignmentPolicy", "legacy") // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") @@ -273,6 +272,7 @@ class GlutenClickHouseNativeWriteTableSuite test("test insert into partition") { withSQLConf( ("spark.gluten.sql.native.writer.enabled", "true"), + ("spark.sql.orc.compression.codec", "lz4"), ("spark.gluten.enabled", "true")) { val originDF = spark.createDataFrame(genTestData()) @@ -310,8 +310,11 @@ class GlutenClickHouseNativeWriteTableSuite + fields.keys.mkString(",") + " from origin_table") - val files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) + var files = recursiveListFiles(new File(getWarehouseDir + "/" + table_name)) .filter(_.getName.endsWith(s".$format")) + if (format == "orc") { + files = files.filter(_.getName.contains(".lz4")) + } assert(files.length == 1) assert(files.head.getAbsolutePath.contains("another_date_field=2020-01-01")) } @@ -565,7 +568,8 @@ class GlutenClickHouseNativeWriteTableSuite } } - test("test hive parquet/orc table, all columns being partitioned. ") { + // This test case will be failed with incorrect result randomly, ignore first. + ignore("test hive parquet/orc table, all columns being partitioned. ") { withSQLConf( ("spark.gluten.sql.native.writer.enabled", "true"), ("spark.gluten.enabled", "true")) { diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala index 2b09b0ca9f93..0774e964e17a 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHousePreferSpillColumnarShuffleAQESuite.scala @@ -18,7 +18,7 @@ package io.glutenproject.execution import org.apache.spark.SparkConf import org.apache.spark.sql.execution.CoalescedPartitionSpec -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ColumnarAQEShuffleReadExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec} class GlutenClickHousePreferSpillColumnarShuffleAQESuite extends GlutenClickHouseTPCHAbstractSuite @@ -46,7 +46,7 @@ class GlutenClickHousePreferSpillColumnarShuffleAQESuite assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) val colCustomShuffleReaderExecs = collect(df.queryExecution.executedPlan) { - case csr: ColumnarAQEShuffleReadExec => csr + case csr: AQEShuffleReadExec => csr } assert(colCustomShuffleReaderExecs.size == 2) val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseSyntheticDataSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseSyntheticDataSuite.scala index 0d4e8f0de95e..761b955d985a 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseSyntheticDataSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseSyntheticDataSuite.scala @@ -77,7 +77,6 @@ class GlutenClickHouseSyntheticDataSuite .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "true") .set("spark.sql.warehouse.dir", warehouse) .set("spark.sql.legacy.createHiveTableByDefault", "false") .set("spark.shuffle.manager", "sort") diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCDSAbstractSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCDSAbstractSuite.scala index 949622193235..42c83cafd05a 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCDSAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCDSAbstractSuite.scala @@ -174,8 +174,8 @@ abstract class GlutenClickHouseTPCDSAbstractSuite .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "true") .set("spark.sql.warehouse.dir", warehouse) + .set("spark.sql.decimalOperations.allowPrecisionLoss", "false") /* .set("spark.sql.catalogImplementation", "hive") .set("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=${ metaStorePathAbsolute + "/metastore_db"};create=true") */ diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHAbstractSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHAbstractSuite.scala index 6e1bc17fce48..12179091dd39 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHAbstractSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHAbstractSuite.scala @@ -518,8 +518,8 @@ abstract class GlutenClickHouseTPCHAbstractSuite .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "true") .set("spark.sql.warehouse.dir", warehouse) + .set("spark.sql.decimalOperations.allowPrecisionLoss", "false") /* .set("spark.sql.catalogImplementation", "hive") .set("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=${ metaStorePathAbsolute + "/metastore_db"};create=true") */ diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala new file mode 100644 index 000000000000..d031e904a7c0 --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHBucketSuite.scala @@ -0,0 +1,488 @@ +/* + * 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 io.glutenproject.execution + +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} +import org.apache.spark.sql.execution.ColumnarInputAdapter +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.commons.io.FileUtils + +import java.io.File + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseTPCHBucketSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val tablesPath: String = basePath + "/tpch-data-ch" + override protected val tpchQueries: String = + rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" + override protected val queriesResults: String = rootPath + "bucket-queries-output" + + protected val bucketTableResourcePath: String = rootPath + "tpch-data-bucket/mergetree_bucket" + protected val bucketTableDataPath: String = basePath + "/tpch-mergetree-bucket" + protected lazy val sparkVersion: String = { + val version = SPARK_VERSION_SHORT.split("\\.") + version(0) + "." + version(1) + } + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.io.compression.codec", "LZ4") + .set("spark.sql.shuffle.partitions", "5") + .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") + } + + override protected val createNullableTables = true + + override def beforeAll(): Unit = { + super.beforeAll() + FileUtils.copyDirectory(new File(bucketTableResourcePath), new File(bucketTableDataPath)) + createTPCHMergeTreeBucketTables() + } + + protected def createTPCHMergeTreeBucketTables(): Unit = { + spark.sql(s""" + |CREATE DATABASE IF NOT EXISTS tpch_mergetree_bucket + |""".stripMargin) + spark.sql("use tpch_mergetree_bucket") + val customerData = bucketTableDataPath + "/customer" + spark.sql(s"DROP TABLE IF EXISTS customer") + // On Spark 3.2, bucket table does not support to create bucket column with sort columns for + // DS V2 + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal double, + | c_mktsegment string, + | c_comment string) + | USING clickhouse + | LOCATION '$customerData' + | CLUSTERED BY (c_custkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (c_custkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val lineitemData = bucketTableDataPath + "/lineitem" + spark.sql(s"DROP TABLE IF EXISTS lineitem") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | 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 '$lineitemData' + | CLUSTERED BY (l_orderkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (l_shipdate, l_orderkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val nationData = bucketTableDataPath + "/nation" + spark.sql(s"DROP TABLE IF EXISTS nation") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING clickhouse + | LOCATION '$nationData' + | CLUSTERED BY (n_nationkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (n_nationkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val regionData = bucketTableDataPath + "/region" + spark.sql(s"DROP TABLE IF EXISTS region") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING clickhouse + | LOCATION '$regionData' + | CLUSTERED BY (r_regionkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (r_regionkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val ordersData = bucketTableDataPath + "/orders" + spark.sql(s"DROP TABLE IF EXISTS orders") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING clickhouse + | LOCATION '$ordersData' + | CLUSTERED BY (o_orderkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (o_orderkey, o_orderdate)"} INTO 2 BUCKETS; + |""".stripMargin) + + val partData = bucketTableDataPath + "/part" + spark.sql(s"DROP TABLE IF EXISTS part") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice double, + | p_comment string) + | USING clickhouse + | LOCATION '$partData' + | CLUSTERED BY (p_partkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (p_partkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val partsuppData = bucketTableDataPath + "/partsupp" + spark.sql(s"DROP TABLE IF EXISTS partsupp") + spark.sql( + s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | USING clickhouse + | LOCATION '$partsuppData' + | CLUSTERED BY (ps_partkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (ps_partkey)"} INTO 2 BUCKETS; + |""".stripMargin) + + val supplierData = bucketTableDataPath + "/supplier" + spark.sql(s"DROP TABLE IF EXISTS supplier") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal double, + | s_comment string) + | USING clickhouse + | LOCATION '$supplierData' + | CLUSTERED BY (s_suppkey) + | ${if (sparkVersion.equals("3.2")) "" else "SORTED BY (s_suppkey)"} INTO 1 BUCKETS; + |""".stripMargin) + + val result = spark + .sql(s""" + | show tables; + |""".stripMargin) + .collect() + assert(result.length == 8) + } + + test("TPCH Q1") { + runTPCHQuery(1)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + } + assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(0).metrics("numFiles").value === 4) + assert(plans(0).metrics("pruningTime").value === -1) + assert(plans(0).metrics("outputRows").value === 591673) + }) + } + + test("TPCH Q2") { + runTPCHQuery(2)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + // Check the bucket join + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + if (sparkVersion.equals("3.2")) { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + } + + if (sparkVersion.equals("3.2")) { + assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(11).metrics("numFiles").value === 1) + assert(plans(11).metrics("outputRows").value === 1000) + }) + } + + test("TPCH Q3") { + runTPCHQuery(3)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + if (sparkVersion.equals("3.2")) { + assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 3111) + + assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(3).metrics("numFiles").value === 4) + assert(plans(3).metrics("outputRows").value === 72678) + }) + } + + test("TPCH Q4") { + runTPCHQuery(4)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 5552) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 379809) + }) + } + + test("TPCH Q6") { + runTPCHQuery(6)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + } + assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(0).metrics("numFiles").value === 4) + assert(plans(0).metrics("pruningTime").value === -1) + assert(plans(0).metrics("outputRows").value === 11618) + }) + } + + test("TPCH Q12") { + runTPCHQuery(12)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 3155) + }) + } + + test("TPCH Q18") { + runTPCHQuery(18)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + // bucket join + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + }) + } + + test("TPCH Q20") { + runTPCHQuery(20)( + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + }) + } +} +// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala index a9ce850a9189..0814c3c8c7d7 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHColumnarShuffleParquetAQESuite.scala @@ -43,6 +43,7 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite .set("spark.sql.autoBroadcastJoinThreshold", "10MB") .set("spark.gluten.sql.columnar.backend.ch.use.v2", "false") .set("spark.sql.adaptive.enabled", "true") + .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") } override protected def createTPCHNotNullTables(): Unit = { @@ -62,7 +63,11 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans(2).metrics("numFiles").value === 1) assert(plans(2).metrics("pruningTime").value === -1) assert(plans(2).metrics("filesSize").value === 17777735) + assert(plans(2).metrics("outputRows").value === 600572) + assert(plans(1).metrics("inputRows").value === 591673) + assert(plans(1).metrics("resizeInputRows").value === 4) + assert(plans(1).metrics("resizeOutputRows").value === 4) assert(plans(1).metrics("outputRows").value === 4) assert(plans(1).metrics("outputVectors").value === 1) @@ -87,6 +92,9 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite assert(plans(2).metrics("pruningTime").value === -1) assert(plans(2).metrics("filesSize").value === 17777735) + assert(plans(1).metrics("inputRows").value === 591673) + assert(plans(1).metrics("resizeInputRows").value === 4) + assert(plans(1).metrics("resizeOutputRows").value === 4) assert(plans(1).metrics("outputRows").value === 4) assert(plans(1).metrics("outputVectors").value === 1) @@ -246,7 +254,17 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite } test("TPCH Q21") { - runTPCHQuery(21, noFallBack = false) { df => } + runTPCHQuery(21, noFallBack = false) { + df => + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case filterExec: FilterExecTransformerBase => filterExec + } + assert(plans(2).metrics("inputRows").value === 600572) + assert(plans(2).metrics("outputRows").value === 379809) + + assert(plans(3).metrics("outputRows").value === 600572) + } } test("TPCH Q22") { @@ -272,4 +290,44 @@ class GlutenClickHouseTPCHColumnarShuffleParquetAQESuite } } } + + test("collect_set") { + val sql = + """ + |select a, b from ( + |select n_regionkey as a, collect_set(if(n_regionkey=0, n_name, null)) + | as set from nation group by n_regionkey) + |lateral view explode(set) as b + |order by a, b + |""".stripMargin + runQueryAndCompare(sql)(checkOperatorMatch[CHHashAggregateExecTransformer]) + } + + test("test 'aggregate function collect_list'") { + val df = runQueryAndCompare( + "select l_orderkey,from_unixtime(l_orderkey, 'yyyy-MM-dd HH:mm:ss') " + + "from lineitem order by l_orderkey desc limit 10" + )(checkOperatorMatch[ProjectExecTransformer]) + checkLengthAndPlan(df, 10) + } + + test("test max string") { + withSQLConf(("spark.gluten.sql.columnar.force.hashagg", "true")) { + val sql = + """ + |SELECT + | l_returnflag, + | l_linestatus, + | max(l_comment) + |FROM + | lineitem + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |""".stripMargin + runQueryAndCompare(sql, noFallBack = false) { df => } + } + } } diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetAQESuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetAQESuite.scala index 3c59f158d0c4..36075a3b7bce 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetAQESuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetAQESuite.scala @@ -46,6 +46,7 @@ class GlutenClickHouseTPCHParquetAQESuite .set("spark.gluten.sql.columnar.backend.ch.use.v2", "false") .set("spark.sql.adaptive.enabled", "true") .set("spark.gluten.sql.columnar.backend.ch.runtime_config.use_local_format", "true") + .set("spark.gluten.sql.columnar.backend.ch.shuffle.hash.algorithm", "sparkMurmurHash3_32") } override protected def createTPCHNotNullTables(): Unit = { @@ -331,5 +332,52 @@ class GlutenClickHouseTPCHParquetAQESuite } ) } + + test("GLUTEN-3534: Fix incorrect logic of judging whether supports pre-project for the shuffle") { + withSQLConf(("spark.sql.autoBroadcastJoinThreshold", "-1")) { + runQueryAndCompare( + s""" + |select t1.l_orderkey, t2.o_orderkey, extract(year from t1.l_shipdate), t2.o_year, + |t1.l_cnt, t2.o_cnt + |from ( + | select l_orderkey, l_shipdate, count(1) as l_cnt + | from lineitem + | group by l_orderkey, l_shipdate) t1 + |join ( + | select o_orderkey, extract(year from o_orderdate) as o_year, count(1) as o_cnt + | from orders + | group by o_orderkey, o_orderdate) t2 + |on t1.l_orderkey = t2.o_orderkey + | and extract(year from t1.l_shipdate) = o_year + |order by t1.l_orderkey, t2.o_orderkey, t2.o_year, t1.l_cnt, t2.o_cnt + |limit 100 + | + |""".stripMargin, + true, + true + )(df => {}) + + runQueryAndCompare( + s""" + |select t1.l_orderkey, t2.o_orderkey, extract(year from t1.l_shipdate), t2.o_year + |from ( + | select l_orderkey, l_shipdate, count(1) as l_cnt + | from lineitem + | group by l_orderkey, l_shipdate) t1 + |join ( + | select o_orderkey, extract(year from o_orderdate) as o_year, count(1) as o_cnt + | from orders + | group by o_orderkey, o_orderdate) t2 + |on t1.l_orderkey = t2.o_orderkey + | and extract(year from t1.l_shipdate) = o_year + |order by t1.l_orderkey, t2.o_orderkey, t2.o_year + |limit 100 + | + |""".stripMargin, + true, + true + )(df => {}) + } + } } // scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala new file mode 100644 index 000000000000..94675988a15c --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetBucketSuite.scala @@ -0,0 +1,502 @@ +/* + * 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 io.glutenproject.execution + +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} +import org.apache.spark.sql.execution.ColumnarInputAdapter +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import org.apache.commons.io.FileUtils + +import java.io.File + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseTPCHParquetBucketSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val resourcePath: String = + "../../../../gluten-core/src/test/resources/tpch-data" + + override protected val tablesPath: String = basePath + "/tpch-data" + override protected val tpchQueries: String = + rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" + override protected val queriesResults: String = rootPath + "queries-output" + + protected val bucketTableResourcePath: String = rootPath + "tpch-data-bucket/parquet_bucket" + protected val bucketTableDataPath: String = basePath + "/tpch-parquet-bucket" + protected lazy val sparkVersion: String = { + val version = SPARK_VERSION_SHORT.split("\\.") + version(0) + "." + version(1) + } + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.io.compression.codec", "LZ4") + .set("spark.sql.shuffle.partitions", "5") + .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") + } + + override protected val createNullableTables = true + + override def beforeAll(): Unit = { + super.beforeAll() + FileUtils.copyDirectory(new File(bucketTableResourcePath), new File(bucketTableDataPath)) + createTPCHParquetBucketTables() + } + + protected def createTPCHParquetBucketTables(): Unit = { + + spark.sql(s""" + |CREATE DATABASE IF NOT EXISTS tpch_parquet_bucket + |""".stripMargin) + spark.sql("use tpch_parquet_bucket") + val customerData = bucketTableDataPath + "/customer" + spark.sql(s"DROP TABLE IF EXISTS customer") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS customer ( + | c_custkey bigint, + | c_name string, + | c_address string, + | c_nationkey bigint, + | c_phone string, + | c_acctbal double, + | c_mktsegment string, + | c_comment string) + | USING PARQUET + | LOCATION '$customerData' + | CLUSTERED BY (c_custkey) SORTED BY (c_custkey) INTO 2 BUCKETS; + |""".stripMargin) + + val lineitemData = bucketTableDataPath + "/lineitem" + spark.sql(s"DROP TABLE IF EXISTS lineitem") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS lineitem ( + | 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 PARQUET + | LOCATION '$lineitemData' + | CLUSTERED BY (l_orderkey) SORTED BY (l_shipdate, l_orderkey) INTO 2 BUCKETS; + |""".stripMargin) + + val nationData = bucketTableDataPath + "/nation" + spark.sql(s"DROP TABLE IF EXISTS nation") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS nation ( + | n_nationkey bigint, + | n_name string, + | n_regionkey bigint, + | n_comment string) + | USING PARQUET + | LOCATION '$nationData' + | CLUSTERED BY (n_nationkey) SORTED BY (n_nationkey) INTO 1 BUCKETS; + |""".stripMargin) + + val regionData = bucketTableDataPath + "/region" + spark.sql(s"DROP TABLE IF EXISTS region") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS region ( + | r_regionkey bigint, + | r_name string, + | r_comment string) + | USING PARQUET + | LOCATION '$regionData' + | CLUSTERED BY (r_regionkey) SORTED BY (r_regionkey) INTO 1 BUCKETS; + |""".stripMargin) + + val ordersData = bucketTableDataPath + "/orders" + spark.sql(s"DROP TABLE IF EXISTS orders") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS orders ( + | o_orderkey bigint, + | o_custkey bigint, + | o_orderstatus string, + | o_totalprice double, + | o_orderdate date, + | o_orderpriority string, + | o_clerk string, + | o_shippriority bigint, + | o_comment string) + | USING PARQUET + | LOCATION '$ordersData' + | CLUSTERED BY (o_orderkey) SORTED BY (o_orderkey, o_orderdate) INTO 2 BUCKETS; + |""".stripMargin) + + val partData = bucketTableDataPath + "/part" + spark.sql(s"DROP TABLE IF EXISTS part") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS part ( + | p_partkey bigint, + | p_name string, + | p_mfgr string, + | p_brand string, + | p_type string, + | p_size bigint, + | p_container string, + | p_retailprice double, + | p_comment string) + | USING PARQUET + | LOCATION '$partData' + | CLUSTERED BY (p_partkey) SORTED BY (p_partkey) INTO 2 BUCKETS; + |""".stripMargin) + + val partsuppData = bucketTableDataPath + "/partsupp" + spark.sql(s"DROP TABLE IF EXISTS partsupp") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS partsupp ( + | ps_partkey bigint, + | ps_suppkey bigint, + | ps_availqty bigint, + | ps_supplycost double, + | ps_comment string) + | USING PARQUET + | LOCATION '$partsuppData' + | CLUSTERED BY (ps_partkey) SORTED BY (ps_partkey) INTO 2 BUCKETS; + |""".stripMargin) + + val supplierData = bucketTableDataPath + "/supplier" + spark.sql(s"DROP TABLE IF EXISTS supplier") + spark.sql(s""" + | CREATE EXTERNAL TABLE IF NOT EXISTS supplier ( + | s_suppkey bigint, + | s_name string, + | s_address string, + | s_nationkey bigint, + | s_phone string, + | s_acctbal double, + | s_comment string) + | USING PARQUET + | LOCATION '$supplierData' + | CLUSTERED BY (s_suppkey) SORTED BY (s_suppkey) INTO 1 BUCKETS; + |""".stripMargin) + + val result = spark + .sql(s""" + | show tables; + |""".stripMargin) + .collect() + assert(result.length == 8) + } + + test("TPCH Q1") { + compareTPCHQueryAgainstVanillaSpark( + 1, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + } + assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(0).metrics("numFiles").value === 4) + assert(plans(0).metrics("pruningTime").value === -1) + assert(plans(0).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q2") { + compareTPCHQueryAgainstVanillaSpark( + 2, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + // Check the bucket join + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(4) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + if (sparkVersion.equals("3.2")) { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(9) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[FilterExecTransformerBase]) + } + + if (sparkVersion.equals("3.2")) { + assert(!(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(11).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(11).metrics("numFiles").value === 1) + assert(plans(11).metrics("outputRows").value === 1000) + } + ) + } + + test("TPCH Q3") { + compareTPCHQueryAgainstVanillaSpark( + 3, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + } + + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + if (sparkVersion.equals("3.2")) { + assert(!(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + } else { + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + } + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 15000) + + assert(!(plans(3).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(3).metrics("numFiles").value === 4) + assert(plans(3).metrics("outputRows").value === 150000) + } + ) + } + + test("TPCH Q4") { + compareTPCHQueryAgainstVanillaSpark( + 4, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ProjectExecTransformer]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q6") { + compareTPCHQueryAgainstVanillaSpark( + 6, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + } + assert(!(plans(0).asInstanceOf[FileSourceScanExecTransformer].bucketedScan)) + assert(plans(0).metrics("numFiles").value === 4) + assert(plans(0).metrics("pruningTime").value === -1) + assert(plans(0).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q12") { + compareTPCHQueryAgainstVanillaSpark( + 12, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case scanExec: BasicScanExecTransformer => scanExec + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(0) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + + assert(plans(1).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(1).metrics("numFiles").value === 4) + assert(plans(1).metrics("outputRows").value === 150000) + + assert(plans(2).asInstanceOf[FileSourceScanExecTransformer].bucketedScan) + assert(plans(2).metrics("numFiles").value === 4) + assert(plans(2).metrics("outputRows").value === 600572) + } + ) + } + + test("TPCH Q18") { + compareTPCHQueryAgainstVanillaSpark( + 18, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + // bucket join + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + // bucket join + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + } + ) + } + + test("TPCH Q20") { + compareTPCHQueryAgainstVanillaSpark( + 20, + tpchQueries, + df => { + val plans = collect(df.queryExecution.executedPlan) { + case joinExec: HashJoinLikeExecTransformer => joinExec + } + if (sparkVersion.equals("3.2")) { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + } else { + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + } + assert( + plans(1) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[ColumnarInputAdapter]) + assert( + plans(2) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ColumnarInputAdapter]) + + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .left + .isInstanceOf[FilterExecTransformerBase]) + assert( + plans(3) + .asInstanceOf[HashJoinLikeExecTransformer] + .right + .isInstanceOf[ProjectExecTransformer]) + } + ) + } +} +// scalastyle:on line.size.limit diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetRFSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetRFSuite.scala index 06d0f41ab7cc..7513e764a6be 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetRFSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetRFSuite.scala @@ -16,13 +16,59 @@ */ package io.glutenproject.execution -import org.apache.spark.SparkConf +import org.apache.spark.{SPARK_VERSION_SHORT, SparkConf} class GlutenClickHouseTPCHParquetRFSuite extends GlutenClickHouseTPCHParquetSuite { + + protected lazy val sparkVersion: String = { + val version = SPARK_VERSION_SHORT.split("\\.") + version(0) + "." + version(1) + } + override protected def sparkConf: SparkConf = { super.sparkConf // radically small threshold to force runtime bloom filter .set("spark.sql.optimizer.runtime.bloomFilter.applicationSideScanSizeThreshold", "1KB") .set("spark.sql.optimizer.runtime.bloomFilter.enabled", "true") } + + test("GLUTEN-3779: Fix core dump when executing sql with runtime filter") { + withSQLConf( + ("spark.sql.autoBroadcastJoinThreshold", "-1"), + ("spark.sql.files.maxPartitionBytes", "204800"), + ("spark.sql.files.openCostInBytes", "102400") + ) { + compareResultsAgainstVanillaSpark( + """ + |SELECT + | sum(l_extendedprice) / 7.0 AS avg_yearly + |FROM + | lineitem, + | part + |WHERE + | p_partkey = l_partkey + | AND p_size > 5 + | AND l_quantity < ( + | SELECT + | 0.2 * avg(l_quantity) + | FROM + | lineitem + | WHERE + | l_partkey = p_partkey); + | + |""".stripMargin, + compareResult = true, + df => { + if (sparkVersion.equals("3.3")) { + val filterExecs = df.queryExecution.executedPlan.collect { + case filter: FilterExecTransformerBase => filter + } + assert(filterExecs.size == 4) + assert( + filterExecs(0).asInstanceOf[FilterExecTransformer].toString.contains("might_contain")) + } + } + ) + } + } } diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetSuite.scala index 435758d68dce..454f29c8bab4 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHParquetSuite.scala @@ -2004,6 +2004,14 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite runQueryAndCompare(sql)(checkOperatorMatch[ProjectExecTransformer]) } + test("GLUTEN-3501: test json output format with struct contains null value") { + val sql = + """ + |select to_json(struct(cast(id as string), null, id, 1.1, 1.1f, 1.1d)) from range(3) + |""".stripMargin + runQueryAndCompare(sql)(checkOperatorMatch[ProjectExecTransformer]) + } + test("GLUTEN-3216: invalid read rel schema in aggregation") { val sql = """ @@ -2013,7 +2021,6 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite } test("Test plan json non-empty") { - spark.sparkContext.setLogLevel("WARN") val df1 = spark .sql(""" | select * from lineitem limit 1 @@ -2021,18 +2028,7 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite val executedPlan1 = df1.queryExecution.executedPlan val lastStageTransformer1 = executedPlan1.find(_.isInstanceOf[WholeStageTransformer]) executedPlan1.execute() - assert(lastStageTransformer1.get.asInstanceOf[WholeStageTransformer].getPlanJson.isEmpty) - - spark.sparkContext.setLogLevel("DEBUG") - val df2 = spark - .sql(""" - | select * from lineitem limit 1 - | """.stripMargin) - val executedPlan2 = df2.queryExecution.executedPlan - val lastStageTransformer2 = executedPlan2.find(_.isInstanceOf[WholeStageTransformer]) - executedPlan2.execute() - assert(lastStageTransformer2.get.asInstanceOf[WholeStageTransformer].getPlanJson.nonEmpty) - spark.sparkContext.setLogLevel(logLevel) + assert(lastStageTransformer1.get.asInstanceOf[WholeStageTransformer].substraitPlanJson.nonEmpty) } test("GLUTEN-3140: Bug fix array_contains return null") { @@ -2060,6 +2056,26 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite compareResultsAgainstVanillaSpark(sql, true, { _ => }) } + test("GLUTEN-3149 convert Inf to int") { + val sql = """ + | select n_regionkey, n is null, isnan(n), cast(n as int) from ( + | select n_regionkey, x, n_regionkey/(x) as n from ( + | select n_regionkey, cast(n_nationkey as float) as x from nation + | )t1 + | )t2""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } + + test("test in-filter contains null value (bigint)") { + val sql = "select s_nationkey from supplier where s_nationkey in (null, 1, 2)" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } + + test("test in-filter contains null value (string)") { + val sql = "select n_name from nation where n_name in ('CANADA', null, 'BRAZIL')" + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } + test("GLUTEN-3287: diff when divide zero") { withSQLConf( SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> (ConstantFolding.ruleName + "," + NullPropagation.ruleName)) { @@ -2087,7 +2103,8 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite |(3, '2023-09-02 00:00:01.333+11'), |(4, '2023-09-02 00:00:01.333-11'), |(5, ' 2023-09-02 agdfegfew'), - |(6, 'afe2023-09-02 11:22:33') + |(6, 'afe2023-09-02 11:22:33'), + |(7, '1970-01-01 00:00:00') |""".stripMargin spark.sql(create_table_sql) spark.sql(insert_data_sql) @@ -2130,5 +2147,75 @@ class GlutenClickHouseTPCHParquetSuite extends GlutenClickHouseTPCHAbstractSuite spark.sql("drop table test_tbl_right_3134") } } + + // Please see the issue: https://github.com/oap-project/gluten/issues/3731 + ignore( + "GLUTEN-3534: Fix incorrect logic of judging whether supports pre-project for the shuffle") { + withSQLConf(("spark.sql.autoBroadcastJoinThreshold", "-1")) { + runQueryAndCompare( + s""" + |select t1.l_orderkey, t2.o_orderkey, extract(year from t1.l_shipdate), t2.o_year, + |t1.l_cnt, t2.o_cnt + |from ( + | select l_orderkey, l_shipdate, count(1) as l_cnt + | from lineitem + | group by l_orderkey, l_shipdate) t1 + |join ( + | select o_orderkey, extract(year from o_orderdate) as o_year, count(1) as o_cnt + | from orders + | group by o_orderkey, o_orderdate) t2 + |on t1.l_orderkey = t2.o_orderkey + | and extract(year from t1.l_shipdate) = o_year + |order by t1.l_orderkey, t2.o_orderkey, t2.o_year, t1.l_cnt, t2.o_cnt + |limit 100 + | + |""".stripMargin, + true, + true + )(df => {}) + + runQueryAndCompare( + s""" + |select t1.l_orderkey, t2.o_orderkey, extract(year from t1.l_shipdate), t2.o_year + |from ( + | select l_orderkey, l_shipdate, count(1) as l_cnt + | from lineitem + | group by l_orderkey, l_shipdate) t1 + |join ( + | select o_orderkey, extract(year from o_orderdate) as o_year, count(1) as o_cnt + | from orders + | group by o_orderkey, o_orderdate) t2 + |on t1.l_orderkey = t2.o_orderkey + | and extract(year from t1.l_shipdate) = o_year + |order by t1.l_orderkey, t2.o_orderkey, t2.o_year + |limit 100 + | + |""".stripMargin, + true, + true + )(df => {}) + } + } + + test("GLUTEN-3467: Fix 'Names of tuple elements must be unique' error for ch backend") { + val sql = + """ + |select named_struct('a', r_regionkey, 'b', r_name, 'a', r_comment) as mergedValue + |from region + |""".stripMargin + compareResultsAgainstVanillaSpark(sql, true, { _ => }) + } + + test("GLUTEN-3521: Bug fix substring index start from 1") { + val tbl_create_sql = "create table test_tbl_3521(id bigint, name string) using parquet"; + val data_insert_sql = "insert into test_tbl_3521 values(1, 'abcdefghijk'), (2, '2023-10-32')"; + val select_sql = + "select id, substring(name, 0), substring(name, 0, 3), substring(name from 0), substring(name from 0 for 100) from test_tbl_3521" + spark.sql(tbl_create_sql) + spark.sql(data_insert_sql) + compareResultsAgainstVanillaSpark(select_sql, true, { _ => }) + spark.sql("drop table test_tbl_3521") + } + } // scalastyle:on line.size.limit diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHSuite.scala index 0cac087b2526..e8b7c1582be2 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseTPCHSuite.scala @@ -444,12 +444,12 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { | create table test_tbl(id bigint, name string) using parquet; |""".stripMargin ) - val df = spark.sql("select count(1), sum(id), max(id), min(id) from test_tbl"); - val result = df.collect() - assert(result(0).getLong(0) == 0) - assert(result(0).isNullAt(1)) - assert(result(0).isNullAt(2)) - assert(result(0).isNullAt(3)) + val sql1 = "select count(1), sum(id), max(id), min(id) from test_tbl" + val sql2 = + "select count(1) as cnt, sum(id) as sum, max(id) as max, min(id) as min from test_tbl" + compareResultsAgainstVanillaSpark(sql1, true, { _ => }) + compareResultsAgainstVanillaSpark(sql2, true, { _ => }) + spark.sql("drop table test_tbl") } test("test 'function json_tuple'") { @@ -474,10 +474,12 @@ class GlutenClickHouseTPCHSuite extends GlutenClickHouseTPCHAbstractSuite { val data_insert_sql = "insert into test_tbl_3271 values(1, 'ab')" val select_sql_1 = "select id, split(data, ',')[1] from test_tbl_3271 where id = 1" val select_sql_2 = "select id, element_at(split(data, ','), 2) from test_tbl_3271 where id = 1" + val select_sql_3 = "select id, element_at(map(id, data), 1) from test_tbl_3271 where id = 1" spark.sql(table_create_sql); spark.sql(data_insert_sql) compareResultsAgainstVanillaSpark(select_sql_1, true, { _ => }) compareResultsAgainstVanillaSpark(select_sql_2, true, { _ => }) + compareResultsAgainstVanillaSpark(select_sql_3, true, { _ => }) spark.sql(table_drop_sql) } diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickhouseFunctionSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickhouseFunctionSuite.scala index f8c5de77446a..30184389706f 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickhouseFunctionSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickhouseFunctionSuite.scala @@ -65,7 +65,6 @@ class GlutenClickhouseFunctionSuite extends GlutenClickHouseTPCHAbstractSuite { .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceshuffledhashjoin", "true") // TODO: support default ANSI policy .set("spark.sql.storeAssignmentPolicy", "legacy") // .set("spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level", "debug") diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenFunctionValidateSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenFunctionValidateSuite.scala index 569c7e5b6fcd..e7e5e317a9b9 100644 --- a/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenFunctionValidateSuite.scala +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/GlutenFunctionValidateSuite.scala @@ -68,7 +68,6 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS .set("spark.gluten.sql.columnar.iterator", "true") .set("spark.gluten.sql.columnar.hashagg.enablefinal", "true") .set("spark.gluten.sql.enable.native.validation", "false") - .set("spark.gluten.sql.columnar.forceshuffledhashjoin", "true") .set("spark.sql.warehouse.dir", warehouse) .set("spark.shuffle.manager", "sort") .set("spark.io.compression.codec", "snappy") @@ -114,26 +113,27 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS val dateSchema = StructType( Array( + StructField("ts", IntegerType, true), StructField("day", DateType, true), StructField("weekday_abbr", StringType, true) ) ) val dateRows = sparkContext.parallelize( Seq( - Row(Date.valueOf("2019-01-01"), "MO"), - Row(Date.valueOf("2019-01-01"), "TU"), - Row(Date.valueOf("2019-01-01"), "TH"), - Row(Date.valueOf("2019-01-01"), "WE"), - Row(Date.valueOf("2019-01-01"), "FR"), - Row(Date.valueOf("2019-01-01"), "SA"), - Row(Date.valueOf("2019-01-01"), "SU"), - Row(Date.valueOf("2019-01-01"), "MO"), - Row(Date.valueOf("2019-01-02"), "MM"), - Row(Date.valueOf("2019-01-03"), "TH"), - Row(Date.valueOf("2019-01-04"), "WE"), - Row(Date.valueOf("2019-01-05"), "FR"), - Row(null, "SA"), - Row(Date.valueOf("2019-01-07"), null) + Row(1546309380, Date.valueOf("2019-01-01"), "MO"), + Row(1546273380, Date.valueOf("2019-01-01"), "TU"), + Row(1546358340, Date.valueOf("2019-01-01"), "TH"), + Row(1546311540, Date.valueOf("2019-01-01"), "WE"), + Row(1546308540, Date.valueOf("2019-01-01"), "FR"), + Row(1546319340, Date.valueOf("2019-01-01"), "SA"), + Row(1546319940, Date.valueOf("2019-01-01"), "SU"), + Row(1546323545, Date.valueOf("2019-01-01"), "MO"), + Row(1546409940, Date.valueOf("2019-01-02"), "MM"), + Row(1546496340, Date.valueOf("2019-01-03"), "TH"), + Row(1546586340, Date.valueOf("2019-01-04"), "WE"), + Row(1546676341, Date.valueOf("2019-01-05"), "FR"), + Row(null, null, "SA"), + Row(1546849141, Date.valueOf("2019-01-07"), null) ) ) val dateTableFile = Files.createTempFile("", ".parquet").toFile @@ -446,4 +446,43 @@ class GlutenFunctionValidateSuite extends GlutenClickHouseWholeStageTransformerS "select cast(concat(cast(id as string), '.1') as int) from range(10)" )(checkOperatorMatch[ProjectExecTransformer]) } + + test("test cast string to float") { + withSQLConf( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + (ConstantFolding.ruleName + "," + NullPropagation.ruleName)) { + runQueryAndCompare( + "select cast('7.921901' as float), cast('7.921901' as double)", + noFallBack = false + )(checkOperatorMatch[ProjectExecTransformer]) + } + } + + test("test round issue: https://github.com/oap-project/gluten/issues/3462") { + runQueryAndCompare( + "select round(0.41875d * id , 4) from range(10);" + )(checkOperatorMatch[ProjectExecTransformer]) + + runQueryAndCompare( + "select round(0.41875f * id , 4) from range(10);" + )(checkOperatorMatch[ProjectExecTransformer]) + } + + test("test date comparision expression override") { + runQueryAndCompare( + "select * from date_table where to_date(from_unixtime(ts)) < '2019-01-02'", + noFallBack = true) { _ => } + runQueryAndCompare( + "select * from date_table where to_date(from_unixtime(ts)) <= '2019-01-02'", + noFallBack = true) { _ => } + runQueryAndCompare( + "select * from date_table where to_date(from_unixtime(ts)) > '2019-01-02'", + noFallBack = true) { _ => } + runQueryAndCompare( + "select * from date_table where to_date(from_unixtime(ts)) >= '2019-01-02'", + noFallBack = true) { _ => } + runQueryAndCompare( + "select * from date_table where to_date(from_unixtime(ts)) = '2019-01-01'", + noFallBack = true) { _ => } + } } diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/CustomAggExpressionTransformer.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/CustomAggExpressionTransformer.scala new file mode 100644 index 000000000000..76ec0d7ad1fb --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/CustomAggExpressionTransformer.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 io.glutenproject.execution.extension + +import io.glutenproject.expression._ +import io.glutenproject.extension.ExpressionExtensionTrait + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.types.{DataType, LongType} + +import scala.collection.mutable.ListBuffer + +case class CustomAggExpressionTransformer() extends ExpressionExtensionTrait { + + lazy val expressionSigs = Seq( + Sig[CustomSum]("custom_sum") + ) + + /** Generate the extension expressions list, format: Sig[XXXExpression]("XXXExpressionName") */ + override def expressionSigList: Seq[Sig] = expressionSigs + + /** Get the attribute index of the extension aggregate functions. */ + override def getAttrsIndexForExtensionAggregateExpr( + aggregateFunc: AggregateFunction, + mode: AggregateMode, + exp: AggregateExpression, + aggregateAttributeList: Seq[Attribute], + aggregateAttr: ListBuffer[Attribute], + resIndex: Int): Int = { + var reIndex = resIndex + aggregateFunc match { + case CustomSum(_, _) => + mode match { + // custom logic: can not support 'Partial' + /* case Partial => + val aggBufferAttr = aggregateFunc.inputAggBufferAttributes + val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr.head) + aggregateAttr += attr + reIndex += 1 + reIndex */ + case Final => + aggregateAttr += aggregateAttributeList(reIndex) + reIndex += 1 + reIndex + case other => + throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") + } + } + } + + /** Get the custom agg function substrait name and the input types of the child */ + override def buildCustomAggregateFunction( + aggregateFunc: AggregateFunction): (Option[String], Seq[DataType]) = { + val substraitAggFuncName = aggregateFunc match { + case customSum: CustomSum => + if (customSum.dataType.isInstanceOf[LongType]) { + Some("custom_sum") + } else { + Some("custom_sum_double") + } + case _ => + extensionExpressionsMapping.get(aggregateFunc.getClass) + } + if (substraitAggFuncName.isEmpty) { + throw new UnsupportedOperationException( + s"Aggregate function ${aggregateFunc.getClass} is not supported.") + } + (substraitAggFuncName, aggregateFunc.children.map(child => child.dataType)) + } +} diff --git a/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/GlutenCustomAggExpressionSuite.scala b/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/GlutenCustomAggExpressionSuite.scala new file mode 100644 index 000000000000..d70010957bec --- /dev/null +++ b/backends-clickhouse/src/test/scala/io/glutenproject/execution/extension/GlutenCustomAggExpressionSuite.scala @@ -0,0 +1,108 @@ +/* + * 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 io.glutenproject.execution.extension + +import io.glutenproject.execution._ +import io.glutenproject.substrait.SubstraitContext +import io.glutenproject.utils.SubstraitPlanPrinterUtil + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.FunctionRegistryBase +import org.apache.spark.sql.catalyst.expressions.aggregate.CustomSum +import org.apache.spark.sql.execution.aggregate.HashAggregateExec + +class GlutenCustomAggExpressionSuite extends GlutenClickHouseTPCHAbstractSuite { + + override protected val resourcePath: String = + "../../../../gluten-core/src/test/resources/tpch-data" + + override protected val tablesPath: String = basePath + "/tpch-data" + override protected val tpchQueries: String = + rootPath + "../../../../gluten-core/src/test/resources/tpch-queries" + override protected val queriesResults: String = rootPath + "queries-output" + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set( + "spark.gluten.sql.columnar.extended.expressions.transformer", + "io.glutenproject.execution.extension.CustomAggExpressionTransformer") + } + + override protected def createTPCHNotNullTables(): Unit = { + createTPCHParquetTables(tablesPath) + } + + override def beforeAll(): Unit = { + super.beforeAll() + val (expressionInfo, builder) = + FunctionRegistryBase.build[CustomSum]("custom_sum", None) + spark.sessionState.functionRegistry.registerFunction( + FunctionIdentifier.apply("custom_sum"), + expressionInfo, + builder + ) + } + + test("test custom aggregate function") { + val sql = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | custom_sum(l_quantity) AS sum_qty, + | custom_sum(l_linenumber) AS sum_linenumber, + | sum(l_extendedprice) AS sum_base_price + |FROM + | lineitem + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + |""".stripMargin + val df = spark.sql(sql) + // Final stage is not supported, it will be fallback + WholeStageTransformerSuite.checkFallBack(df, false) + + val planExecs = df.queryExecution.executedPlan.collect { + case agg: HashAggregateExec => agg + case aggTransformer: HashAggregateExecBaseTransformer => aggTransformer + case wholeStage: WholeStageTransformer => wholeStage + } + + // First stage fallback + assert(planExecs(3).isInstanceOf[HashAggregateExec]) + + val substraitContext = new SubstraitContext + planExecs(2).asInstanceOf[CHHashAggregateExecTransformer].doTransform(substraitContext) + + // Check the functions + assert(substraitContext.registeredFunction.containsKey("custom_sum_double:req_fp64")) + assert(substraitContext.registeredFunction.containsKey("custom_sum:req_i64")) + assert(substraitContext.registeredFunction.containsKey("sum:req_fp64")) + + val wx = planExecs(1).asInstanceOf[WholeStageTransformer].doWholeStageTransform() + val planJson = SubstraitPlanPrinterUtil.substraitPlanToJson(wx.root.toProtobuf) + assert(planJson.contains("#Partial#custom_sum_double")) + assert(planJson.contains("#Partial#custom_sum")) + assert(planJson.contains("#Partial#sum")) + } +} diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CustomSum.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CustomSum.scala new file mode 100644 index 000000000000..c2d7cf622b3d --- /dev/null +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CustomSum.scala @@ -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. + */ +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.trees.TreePattern.{SUM, TreePattern} +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Port the `Sum` aggregate function from Vanilla Spark 3.2, only changes class name to CustomSum. + */ +case class CustomSum(child: Expression, failOnError: Boolean = SQLConf.get.ansiEnabled) + extends DeclarativeAggregate + with ImplicitCastInputTypes + with UnaryLike[Expression] { + + def this(child: Expression) = this(child, failOnError = SQLConf.get.ansiEnabled) + + override def nullable: Boolean = true + + // Return data type. + override def dataType: DataType = resultType + + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(NumericType, YearMonthIntervalType, DayTimeIntervalType)) + + override def checkInputDataTypes(): TypeCheckResult = + TypeUtils.checkForAnsiIntervalOrNumericType(child.dataType, "sum") + + final override val nodePatterns: Seq[TreePattern] = Seq(SUM) + + private lazy val resultType = child.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType.bounded(precision + 10, scale) + case _: IntegralType => LongType + case it: YearMonthIntervalType => it + case it: DayTimeIntervalType => it + case _ => DoubleType + } + + private lazy val sum = AttributeReference("sum", resultType)() + + private lazy val isEmpty = AttributeReference("isEmpty", BooleanType, nullable = false)() + + private lazy val zero = Literal.default(resultType) + + override lazy val aggBufferAttributes = resultType match { + case _: DecimalType => sum :: isEmpty :: Nil + case _ => sum :: Nil + } + + override lazy val initialValues: Seq[Expression] = resultType match { + case _: DecimalType => Seq(zero, Literal(true, BooleanType)) + case _ => Seq(Literal(null, resultType)) + } + + override lazy val updateExpressions: Seq[Expression] = { + resultType match { + case _: DecimalType => + // For decimal type, the initial value of `sum` is 0. We need to keep `sum` unchanged if + // the input is null, as SUM function ignores null input. The `sum` can only be null if + // overflow happens under non-ansi mode. + val sumExpr = if (child.nullable) { + If(child.isNull, sum, sum + KnownNotNull(child).cast(resultType)) + } else { + sum + child.cast(resultType) + } + // The buffer becomes non-empty after seeing the first not-null input. + val isEmptyExpr = if (child.nullable) { + isEmpty && child.isNull + } else { + Literal(false, BooleanType) + } + Seq(sumExpr, isEmptyExpr) + case _ => + // For non-decimal type, the initial value of `sum` is null, which indicates no value. + // We need `coalesce(sum, zero)` to start summing values. And we need an outer `coalesce` + // in case the input is nullable. The `sum` can only be null if there is no value, as + // non-decimal type can produce overflowed value under non-ansi mode. + if (child.nullable) { + Seq(coalesce(coalesce(sum, zero) + child.cast(resultType), sum)) + } else { + Seq(coalesce(sum, zero) + child.cast(resultType)) + } + } + } + + /** + * For decimal type: If isEmpty is false and if sum is null, then it means we have had an + * overflow. + * + * update of the sum is as follows: Check if either portion of the left.sum or right.sum has + * overflowed If it has, then the sum value will remain null. If it did not have overflow, then + * add the sum.left and sum.right + * + * isEmpty: Set to false if either one of the left or right is set to false. This means we have + * seen atleast a value that was not null. + */ + override lazy val mergeExpressions: Seq[Expression] = { + resultType match { + case _: DecimalType => + val bufferOverflow = !isEmpty.left && sum.left.isNull + val inputOverflow = !isEmpty.right && sum.right.isNull + Seq( + If( + bufferOverflow || inputOverflow, + Literal.create(null, resultType), + // If both the buffer and the input do not overflow, just add them, as they can't be + // null. See the comments inside `updateExpressions`: `sum` can only be null if + // overflow happens. + KnownNotNull(sum.left) + KnownNotNull(sum.right) + ), + isEmpty.left && isEmpty.right + ) + case _ => Seq(coalesce(coalesce(sum.left, zero) + sum.right, sum.left)) + } + } + + /** + * If the isEmpty is true, then it means there were no values to begin with or all the values were + * null, so the result will be null. If the isEmpty is false, then if sum is null that means an + * overflow has happened. So now, if ansi is enabled, then throw exception, if not then return + * null. If sum is not null, then return the sum. + */ + override lazy val evaluateExpression: Expression = resultType match { + case d: DecimalType => + If(isEmpty, Literal.create(null, resultType), CheckOverflowInSum(sum, d, !failOnError)) + case _ => sum + } + + override protected def withNewChildInternal(newChild: Expression): CustomSum = + copy(child = newChild) + + // The flag `failOnError` won't be shown in the `toString` or `toAggString` methods + override def flatArguments: Iterator[Any] = Iterator(child) +} diff --git a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHParquetReadBenchmark.scala b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHParquetReadBenchmark.scala index 54f0e19c0e4c..e6c3d467d0f2 100644 --- a/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHParquetReadBenchmark.scala +++ b/backends-clickhouse/src/test/scala/org/apache/spark/sql/execution/benchmarks/CHParquetReadBenchmark.scala @@ -115,8 +115,7 @@ object CHParquetReadBenchmark extends SqlBasedBenchmark with CHSqlBasedBenchmark val nativeFileScanRDD = BackendsApiManager.getIteratorApiInstance.genNativeFileScanRDD( spark.sparkContext, WholeStageTransformContext(planNode, substraitContext), - fileFormat, - filePartitions, + chFileScan.getSplitInfos, numOutputRows, numOutputVectors, scanTime diff --git a/backends-velox/pom.xml b/backends-velox/pom.xml index 3a42df280f2a..10ff8909b757 100755 --- a/backends-velox/pom.xml +++ b/backends-velox/pom.xml @@ -55,15 +55,9 @@ spark-catalyst_${scala.binary.version} provided - - org.apache.spark - spark-hive-thriftserver_${scala.binary.version} - provided - org.apache.spark spark-hive_${scala.binary.version} - ${spark.version} provided diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/IteratorApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/IteratorApiImpl.scala index a77d8163e4b2..49950669786d 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/IteratorApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/IteratorApiImpl.scala @@ -21,11 +21,12 @@ import io.glutenproject.backendsapi.IteratorApi import io.glutenproject.execution._ import io.glutenproject.metrics.IMetrics import io.glutenproject.substrait.plan.PlanNode -import io.glutenproject.substrait.rel.LocalFilesBuilder +import io.glutenproject.substrait.rel.{LocalFilesBuilder, SplitInfo} import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat +import io.glutenproject.utils.Iterators import io.glutenproject.vectorized._ -import org.apache.spark.{InterruptibleIterator, Partition, SparkConf, SparkContext, TaskContext} +import org.apache.spark.{SparkConf, SparkContext, TaskContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -39,16 +40,16 @@ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.{BinaryType, DateType, StructType, TimestampType} import org.apache.spark.sql.utils.OASPackageBridge.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.{ExecutorManager, TaskResources} +import org.apache.spark.util.ExecutorManager +import java.lang.{Long => JLong} import java.net.URLDecoder import java.nio.charset.StandardCharsets import java.time.ZoneOffset -import java.util +import java.util.{ArrayList => JArrayList, HashMap => JHashMap, Map => JMap} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import scala.collection.mutable class IteratorApiImpl extends IteratorApi with Logging { @@ -57,82 +58,63 @@ class IteratorApiImpl extends IteratorApi with Logging { * * @return */ - override def genFilePartition( - index: Int, - partitions: Seq[InputPartition], - partitionSchemas: Seq[StructType], - fileFormats: Seq[ReadFileFormat], - wsCxt: WholeStageTransformContext): BaseGlutenPartition = { - - def constructSplitInfo(schema: StructType, files: Array[PartitionedFile]) = { - val paths = mutable.ArrayBuffer.empty[String] - val starts = mutable.ArrayBuffer.empty[java.lang.Long] - val lengths = mutable.ArrayBuffer.empty[java.lang.Long] - val partitionColumns = mutable.ArrayBuffer.empty[Map[String, String]] - files.foreach { - file => - paths.append(URLDecoder.decode(file.filePath, StandardCharsets.UTF_8.name())) - starts.append(java.lang.Long.valueOf(file.start)) - lengths.append(java.lang.Long.valueOf(file.length)) + override def genSplitInfo( + partition: InputPartition, + partitionSchemas: StructType, + fileFormat: ReadFileFormat): SplitInfo = { + partition match { + case f: FilePartition => + val (paths, starts, lengths, partitionColumns) = + constructSplitInfo(partitionSchemas, f.files) + val preferredLocations = + SoftAffinityUtil.getFilePartitionLocations(paths.asScala.toArray, f.preferredLocations()) + LocalFilesBuilder.makeLocalFiles( + f.index, + paths, + starts, + lengths, + partitionColumns, + fileFormat, + preferredLocations.toList.asJava) + case _ => + throw new UnsupportedOperationException(s"Unsupported input partition.") + } + } - val partitionColumn = mutable.Map.empty[String, String] - for (i <- 0 until file.partitionValues.numFields) { - val partitionColumnValue = if (file.partitionValues.isNullAt(i)) { - ExternalCatalogUtils.DEFAULT_PARTITION_NAME - } else { - val pn = file.partitionValues.get(i, schema.fields(i).dataType) - schema.fields(i).dataType match { - case _: BinaryType => - new String(pn.asInstanceOf[Array[Byte]], StandardCharsets.UTF_8) - case _: DateType => - DateFormatter.apply().format(pn.asInstanceOf[Integer]) - case _: TimestampType => - TimestampFormatter - .getFractionFormatter(ZoneOffset.UTC) - .format(pn.asInstanceOf[java.lang.Long]) - case _ => pn.toString - } + private def constructSplitInfo(schema: StructType, files: Array[PartitionedFile]) = { + val paths = new JArrayList[String]() + val starts = new JArrayList[JLong] + val lengths = new JArrayList[JLong]() + val partitionColumns = new JArrayList[JMap[String, String]] + files.foreach { + file => + paths.add(URLDecoder.decode(file.filePath.toString, StandardCharsets.UTF_8.name())) + starts.add(JLong.valueOf(file.start)) + lengths.add(JLong.valueOf(file.length)) + + val partitionColumn = new JHashMap[String, String]() + for (i <- 0 until file.partitionValues.numFields) { + val partitionColumnValue = if (file.partitionValues.isNullAt(i)) { + ExternalCatalogUtils.DEFAULT_PARTITION_NAME + } else { + val pn = file.partitionValues.get(i, schema.fields(i).dataType) + schema.fields(i).dataType match { + case _: BinaryType => + new String(pn.asInstanceOf[Array[Byte]], StandardCharsets.UTF_8) + case _: DateType => + DateFormatter.apply().format(pn.asInstanceOf[Integer]) + case _: TimestampType => + TimestampFormatter + .getFractionFormatter(ZoneOffset.UTC) + .format(pn.asInstanceOf[java.lang.Long]) + case _ => pn.toString } - partitionColumn.put(schema.names(i), partitionColumnValue) } - partitionColumns.append(partitionColumn.toMap) - } - (paths, starts, lengths, partitionColumns) + partitionColumn.put(schema.names(i), partitionColumnValue) + } + partitionColumns.add(partitionColumn) } - - val localFilesNodesWithLocations = partitions.indices.map( - i => - partitions(i) match { - case f: FilePartition => - val fileFormat = fileFormats(i) - val partitionSchema = partitionSchemas(i) - val (paths, starts, lengths, partitionColumns) = - constructSplitInfo(partitionSchema, f.files) - ( - LocalFilesBuilder.makeLocalFiles( - f.index, - paths.asJava, - starts.asJava, - lengths.asJava, - partitionColumns.map(_.asJava).asJava, - fileFormat), - SoftAffinityUtil.getFilePartitionLocations(f)) - }) - wsCxt.substraitContext.initLocalFilesNodesIndex(0) - wsCxt.substraitContext.setLocalFilesNodes(localFilesNodesWithLocations.map(_._1)) - val substraitPlan = wsCxt.root.toProtobuf - GlutenPartition(index, substraitPlan, localFilesNodesWithLocations.head._2) - } - - /** - * Generate closeable ColumnBatch iterator. - * - * @param iter - * @return - */ - override def genCloseableColumnBatchIterator( - iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] = { - new CloseableColumnBatchIterator(iter) + (paths, starts, lengths, partitionColumns) } /** @@ -149,35 +131,26 @@ class IteratorApiImpl extends IteratorApi with Logging { inputIterators: Seq[Iterator[ColumnarBatch]] = Seq()): Iterator[ColumnarBatch] = { val beforeBuild = System.nanoTime() val columnarNativeIterators = - new util.ArrayList[GeneralInIterator](inputIterators.map { + new JArrayList[GeneralInIterator](inputIterators.map { iter => new ColumnarBatchInIterator(iter.asJava) }.asJava) val transKernel = NativePlanEvaluator.create() val resIter: GeneralOutIterator = transKernel.createKernelWithBatchIterator(inputPartition.plan, columnarNativeIterators) pipelineTime += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - beforeBuild) - TaskResources.addRecycler(s"FirstStageIterator_${resIter.getId}", 100)(resIter.close()) - val iter = new Iterator[ColumnarBatch] { - private val inputMetrics = TaskContext.get().taskMetrics().inputMetrics - - override def hasNext: Boolean = { - val res = resIter.hasNext - if (!res) { - updateNativeMetrics(resIter.getMetrics) - updateInputMetrics(inputMetrics) - } - res - } - override def next(): ColumnarBatch = { - if (!hasNext) { - throw new java.util.NoSuchElementException("End of stream") - } - resIter.next() + Iterators + .wrap(resIter.asScala) + .protectInvocationFlow() + .recycleIterator { + updateNativeMetrics(resIter.getMetrics) + updateInputMetrics(TaskContext.get().taskMetrics().inputMetrics) + resIter.close() } - } - - new InterruptibleIterator(context, new CloseableColumnBatchIterator(iter, Some(pipelineTime))) + .recyclePayload(batch => batch.close()) + .addToPipelineTime(pipelineTime) + .asInterruptible(context) + .create() } // scalastyle:off argcount @@ -203,33 +176,25 @@ class IteratorApiImpl extends IteratorApi with Logging { val transKernel = NativePlanEvaluator.create() val columnarNativeIterator = - new util.ArrayList[GeneralInIterator](inputIterators.map { + new JArrayList[GeneralInIterator](inputIterators.map { iter => new ColumnarBatchInIterator(iter.asJava) }.asJava) val nativeResultIterator = - transKernel.createKernelWithBatchIterator(rootNode.toProtobuf, columnarNativeIterator) + transKernel.createKernelWithBatchIterator( + rootNode.toProtobuf.toByteArray, + columnarNativeIterator) pipelineTime += TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - beforeBuild) - val resIter = new Iterator[ColumnarBatch] { - override def hasNext: Boolean = { - val res = nativeResultIterator.hasNext - if (!res) { - updateNativeMetrics(nativeResultIterator.getMetrics) - } - res + Iterators + .wrap(nativeResultIterator.asScala) + .recycleIterator { + updateNativeMetrics(nativeResultIterator.getMetrics) + nativeResultIterator.close() } - - override def next(): ColumnarBatch = { - nativeResultIterator.next - } - } - - TaskResources.addRecycler(s"FinalStageIterator_${nativeResultIterator.getId}", 100) { - nativeResultIterator.close() - } - - new CloseableColumnBatchIterator(resIter, Some(pipelineTime)) + .recyclePayload(batch => batch.close()) + .addToPipelineTime(pipelineTime) + .create() } // scalastyle:on argcount @@ -237,8 +202,7 @@ class IteratorApiImpl extends IteratorApi with Logging { override def genNativeFileScanRDD( sparkContext: SparkContext, wsCxt: WholeStageTransformContext, - fileFormat: ReadFileFormat, - inputPartitions: Seq[InputPartition], + splitInfos: Seq[SplitInfo], numOutputRows: SQLMetric, numOutputBatches: SQLMetric, scanTime: SQLMetric): RDD[ColumnarBatch] = { @@ -247,11 +211,12 @@ class IteratorApiImpl extends IteratorApi with Logging { /** Compute for BroadcastBuildSideRDD */ override def genBroadcastBuildSideIterator( - split: Partition, - context: TaskContext, broadcasted: Broadcast[BuildSideRelation], broadCastContext: BroadCastHashJoinContext): Iterator[ColumnarBatch] = { val relation = broadcasted.value.asReadOnlyCopy(broadCastContext) - new CloseableColumnBatchIterator(relation.deserialized) + Iterators + .wrap(relation.deserialized) + .recyclePayload(batch => batch.close()) + .create() } } diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ListenerApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ListenerApiImpl.scala index 04495d5a4070..500178803f5a 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ListenerApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ListenerApiImpl.scala @@ -35,10 +35,11 @@ import org.apache.commons.lang3.StringUtils import scala.sys.process._ class ListenerApiImpl extends ListenerApi { + private val ARROW_VERSION = "1400" override def onDriverStart(conf: SparkConf): Unit = { // sql table cache serializer - if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, defaultValue = true)) { + if (conf.getBoolean(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, defaultValue = false)) { conf.set( StaticSQLConf.SPARK_CACHE_SERIALIZER.key, "org.apache.spark.sql.execution.ColumnarCachedBatchSerializer") @@ -101,16 +102,25 @@ class ListenerApiImpl extends ListenerApi { } loader .newTransaction() - .loadAndCreateLink("libarrow.so.1200.0.0", "libarrow.so.1200", false) - .loadAndCreateLink("libparquet.so.1200.0.0", "libparquet.so.1200", false) + .loadAndCreateLink(s"libarrow.so.$ARROW_VERSION.1.0", s"libarrow.so.$ARROW_VERSION", false) + .loadAndCreateLink( + s"libparquet.so.$ARROW_VERSION.1.0", + s"libparquet.so.$ARROW_VERSION", + false) .commit() } private def loadLibWithMacOS(loader: JniLibLoader): Unit = { loader .newTransaction() - .loadAndCreateLink("libarrow.1200.0.0.dylib", "libarrow.1200.dylib", false) - .loadAndCreateLink("libparquet.1200.0.0.dylib", "libparquet.1200.dylib", false) + .loadAndCreateLink( + s"libarrow.$ARROW_VERSION.1.0.dylib", + s"libarrow.$ARROW_VERSION.dylib", + false) + .loadAndCreateLink( + s"libparquet.$ARROW_VERSION.1.0.dylib", + s"libparquet.$ARROW_VERSION.dylib", + false) .commit() } diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala index cb72ab5caeeb..95f4ec30a5f6 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/MetricsApiImpl.scala @@ -25,14 +25,15 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import java.{lang, util} +import java.lang.{Long => JLong} +import java.util.{List => JList, Map => JMap} class MetricsApiImpl extends MetricsApi with Logging { override def metricsUpdatingFunction( child: SparkPlan, - relMap: util.HashMap[lang.Long, util.ArrayList[lang.Long]], - joinParamsMap: util.HashMap[lang.Long, JoinParams], - aggParamsMap: util.HashMap[lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { MetricsUtil.updateNativeMetrics(child, relMap, joinParamsMap, aggParamsMap) } diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/SparkPlanExecApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/SparkPlanExecApiImpl.scala index 697739ef3d62..db77ce2ae93e 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/SparkPlanExecApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/SparkPlanExecApiImpl.scala @@ -56,6 +56,9 @@ import org.apache.commons.lang3.ClassUtils import javax.ws.rs.core.UriBuilder +import java.lang.{Long => JLong} +import java.util.{Map => JMap} + import scala.collection.mutable.ArrayBuffer class SparkPlanExecApiImpl extends SparkPlanExecApi { @@ -67,7 +70,7 @@ class SparkPlanExecApiImpl extends SparkPlanExecApi { */ override def genGetArrayItemExpressionNode( substraitExprName: String, - functionMap: java.util.HashMap[String, java.lang.Long], + functionMap: JMap[String, JLong], leftNode: ExpressionNode, rightNode: ExpressionNode, original: GetArrayItem): ExpressionNode = { diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala index 263f4a710b31..61ad92b6b931 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/TransformerApiImpl.scala @@ -17,9 +17,12 @@ package io.glutenproject.backendsapi.velox import io.glutenproject.backendsapi.TransformerApi +import io.glutenproject.exec.Runtimes +import io.glutenproject.expression.ConverterUtils import io.glutenproject.extension.ValidationResult -import io.glutenproject.substrait.expression.ExpressionNode +import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} import io.glutenproject.utils.InputPartitionsUtil +import io.glutenproject.vectorized.PlanEvaluatorJniWrapper import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateMap, Explode, Generator, JsonTuple, Literal, PosExplode} @@ -30,7 +33,7 @@ import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDi import org.apache.spark.sql.types._ import org.apache.spark.util.collection.BitSet -import java.util +import java.util.{Map => JMap} class TransformerApiImpl extends TransformerApi with Logging { @@ -50,6 +53,7 @@ class TransformerApiImpl extends TransformerApi with Logging { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] = { @@ -57,6 +61,7 @@ class TransformerApiImpl extends TransformerApi with Logging { relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) @@ -64,7 +69,7 @@ class TransformerApiImpl extends TransformerApi with Logging { } override def postProcessNativeConfig( - nativeConfMap: util.Map[String, String], + nativeConfMap: JMap[String, String], backendPrefix: String): Unit = { // TODO: IMPLEMENT SPECIAL PROCESS FOR VELOX BACKEND } @@ -100,7 +105,7 @@ class TransformerApiImpl extends TransformerApi with Logging { override def createDateDiffParamList( start: ExpressionNode, end: ExpressionNode): Iterable[ExpressionNode] = { - List(start, end) + List(end, start) } override def createLikeParamList( @@ -109,4 +114,25 @@ class TransformerApiImpl extends TransformerApi with Logging { escapeChar: ExpressionNode): Iterable[ExpressionNode] = { List(left, right, escapeChar) } + + override def createCheckOverflowExprNode( + args: java.lang.Object, + substraitExprName: String, + childNode: ExpressionNode, + dataType: DecimalType, + nullable: Boolean, + nullOnOverflow: Boolean): ExpressionNode = { + val typeNode = ConverterUtils.getTypeNode(dataType, nullable) + ExpressionBuilder.makeCast(typeNode, childNode, !nullOnOverflow) + } + + override def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String = { + val tmpRuntime = Runtimes.tmpInstance() + try { + val jniWrapper = PlanEvaluatorJniWrapper.forRuntime(tmpRuntime) + jniWrapper.nativePlanString(substraitPlan, details) + } finally { + tmpRuntime.release() + } + } } diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ValidatorApiImpl.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ValidatorApiImpl.scala index 6d606cf52310..4d64a790c8d1 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ValidatorApiImpl.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/ValidatorApiImpl.scala @@ -17,6 +17,7 @@ package io.glutenproject.backendsapi.velox import io.glutenproject.backendsapi.ValidatorApi +import io.glutenproject.exec.Runtimes import io.glutenproject.substrait.plan.PlanNode import io.glutenproject.validate.NativePlanValidationInfo import io.glutenproject.vectorized.NativePlanEvaluator @@ -31,8 +32,13 @@ class ValidatorApiImpl extends ValidatorApi { doExprValidate(Map(), substraitExprName, expr) override def doNativeValidateWithFailureReason(plan: PlanNode): NativePlanValidationInfo = { - val validator = NativePlanEvaluator.createForValidation() - validator.doNativeValidateWithFailureReason(plan.toProtobuf.toByteArray) + val tmpRuntime = Runtimes.tmpInstance() + try { + val validator = NativePlanEvaluator.createForValidation(tmpRuntime) + validator.doNativeValidateWithFailureReason(plan.toProtobuf.toByteArray) + } finally { + tmpRuntime.release() + } } override def doSparkPlanValidate(plan: SparkPlan): Boolean = true diff --git a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala index 2790820dd8da..40d42b90873e 100644 --- a/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala +++ b/backends-velox/src/main/scala/io/glutenproject/backendsapi/velox/VeloxBackend.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.command.CreateDataSourceTableAsSelectCommand import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.expression.UDFResolver +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import scala.util.control.Breaks.breakable @@ -305,8 +306,6 @@ object BackendSettings extends BackendSettingsApi { UDFResolver.resolveUdfConf(nativeConf) } - override def supportBucketScan(): Boolean = true - override def insertPostProjectForGenerate(): Boolean = true override def skipNativeCtas(ctas: CreateDataSourceTableAsSelectCommand): Boolean = true @@ -322,4 +321,6 @@ object BackendSettings extends BackendSettingsApi { override def requiredChildOrderingForWindow(): Boolean = true override def staticPartitionWriteOnly(): Boolean = true + + override def allowDecimalArithmetic: Boolean = SQLConf.get.decimalOperationsAllowPrecisionLoss } diff --git a/backends-velox/src/main/scala/io/glutenproject/execution/FilterExecTransformer.scala b/backends-velox/src/main/scala/io/glutenproject/execution/FilterExecTransformer.scala index 41864e07dba1..426132da70ed 100644 --- a/backends-velox/src/main/scala/io/glutenproject/execution/FilterExecTransformer.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/FilterExecTransformer.scala @@ -28,8 +28,7 @@ import java.util import scala.collection.JavaConverters._ case class FilterExecTransformer(condition: Expression, child: SparkPlan) - extends FilterExecTransformerBase(condition, child) - with TransformSupport { + extends FilterExecTransformerBase(condition, child) { override protected def doValidateInternal(): ValidationResult = { val leftCondition = getLeftCondition diff --git a/backends-velox/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala b/backends-velox/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala index 3d842ab25761..2d12eae0d41f 100644 --- a/backends-velox/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/HashAggregateExecTransformer.scala @@ -16,7 +16,6 @@ */ package io.glutenproject.execution -import io.glutenproject.execution.VeloxAggregateFunctionsBuilder.{veloxFourIntermediateTypes, veloxSixIntermediateTypes, veloxThreeIntermediateTypes} import io.glutenproject.expression._ import io.glutenproject.expression.ConverterUtils.FunctionConfig import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} @@ -24,7 +23,7 @@ import io.glutenproject.substrait.{AggregationParams, SubstraitContext} import io.glutenproject.substrait.expression.{AggregateFunctionNode, ExpressionBuilder, ExpressionNode, ScalarFunctionNode} import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} -import io.glutenproject.utils.GlutenDecimalUtil +import io.glutenproject.utils.VeloxIntermediateData import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -34,10 +33,10 @@ import org.apache.spark.sql.types._ import com.google.protobuf.Any -import java.util +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, HashMap => JHashMap, List => JList} import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer case class HashAggregateExecTransformer( requiredChildDistributionExpressions: Option[Seq[Expression]], @@ -56,34 +55,18 @@ case class HashAggregateExecTransformer( resultExpressions, child) { - override protected def getAttrForAggregateExpr( - exp: AggregateExpression, - aggregateAttributeList: Seq[Attribute], - aggregateAttr: ListBuffer[Attribute], - index: Int): Int = { - var resIndex = index - val mode = exp.mode - val aggregateFunc = exp.aggregateFunction - aggregateFunc match { - case hllAdapter: HLLAdapter => + override protected def checkAggFuncModeSupport( + aggFunc: AggregateFunction, + mode: AggregateMode): Boolean = { + aggFunc match { + case _: HLLAdapter => mode match { - case Partial => - val aggBufferAttr = hllAdapter.inputAggBufferAttributes - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += aggBufferAttr.size - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"not currently supported: $other.") + case Partial | Final => true + case _ => false } case _ => - resIndex = super.getAttrForAggregateExpr(exp, aggregateAttributeList, aggregateAttr, index) + super.checkAggFuncModeSupport(aggFunc, mode) } - resIndex } override protected def withNewChildInternal(newChild: SparkPlan): HashAggregateExecTransformer = { @@ -96,27 +79,18 @@ case class HashAggregateExecTransformer( * @return * extracting needed or not. */ - def extractStructNeeded(): Boolean = { - for (expr <- aggregateExpressions) { - val aggregateFunction = expr.aggregateFunction - aggregateFunction match { - case _: Average | _: First | _: Last | _: StddevSamp | _: StddevPop | _: VarianceSamp | - _: VariancePop | _: Corr | _: CovPopulation | _: CovSample => - expr.mode match { - case Partial | PartialMerge => - return true - case _ => - } - case sum: Sum if sum.dataType.isInstanceOf[DecimalType] => - expr.mode match { - case Partial | PartialMerge => - return true - case _ => - } - case _ => - } + private def extractStructNeeded(): Boolean = { + aggregateExpressions.exists { + expr => + expr.aggregateFunction match { + case aggFunc if aggFunc.aggBufferAttributes.size > 1 => + expr.mode match { + case Partial | PartialMerge => true + case _ => false + } + case _ => false + } } - false } /** @@ -135,7 +109,7 @@ case class HashAggregateExecTransformer( aggRel: RelNode, operatorId: Long, validation: Boolean): RelNode = { - val expressionNodes = new util.ArrayList[ExpressionNode]() + val expressionNodes = new JArrayList[ExpressionNode]() var colIdx = 0 while (colIdx < groupingExpressions.size) { val groupingExpr: ExpressionNode = ExpressionBuilder.makeSelection(colIdx) @@ -149,56 +123,29 @@ case class HashAggregateExecTransformer( case _ => throw new UnsupportedOperationException(s"${expr.mode} not supported.") } + val aggFunc = expr.aggregateFunction expr.aggregateFunction match { - case _: Average | _: First | _: Last => - // Select first and second aggregate buffer from Velox Struct. - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 0)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 1)) - colIdx += 1 - case _: StddevSamp | _: StddevPop | _: VarianceSamp | _: VariancePop => - // Select count from Velox struct with count casted from LongType into DoubleType. - expressionNodes.add( - ExpressionBuilder - .makeCast( - ConverterUtils.getTypeNode(DoubleType, nullable = false), - ExpressionBuilder.makeSelection(colIdx, 0), - SQLConf.get.ansiEnabled)) - // Select avg from Velox Struct. - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 1)) - // Select m2 from Velox Struct. - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 2)) - colIdx += 1 - case sum: Sum if sum.dataType.isInstanceOf[DecimalType] => - // Select sum from Velox Struct. - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 0)) - // Select isEmpty from Velox Struct. - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 1)) - colIdx += 1 - case _: Corr => - // Select count from Velox struct with count casted from LongType into DoubleType. - expressionNodes.add( - ExpressionBuilder - .makeCast( - ConverterUtils.getTypeNode(DoubleType, nullable = false), - ExpressionBuilder.makeSelection(colIdx, 1), - SQLConf.get.ansiEnabled)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 4)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 5)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 0)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 2)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 3)) - colIdx += 1 - case _: CovPopulation | _: CovSample => - // Select count from Velox struct with count casted from LongType into DoubleType. - expressionNodes.add( - ExpressionBuilder - .makeCast( - ConverterUtils.getTypeNode(DoubleType, nullable = false), - ExpressionBuilder.makeSelection(colIdx, 1), - SQLConf.get.ansiEnabled)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 2)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 3)) - expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, 0)) + case _ @VeloxIntermediateData.Type(veloxTypes: Seq[DataType]) => + val (sparkOrders, sparkTypes) = + aggFunc.aggBufferAttributes.map(attr => (attr.name, attr.dataType)).unzip + val veloxOrders = VeloxIntermediateData.veloxIntermediateDataOrder(aggFunc) + val adjustedOrders = sparkOrders.map(veloxOrders.indexOf(_)) + sparkTypes.zipWithIndex.foreach { + case (sparkType, idx) => + val veloxType = veloxTypes(adjustedOrders(idx)) + if (veloxType != sparkType) { + // Velox and Spark have different type, adding a cast expression + expressionNodes.add( + ExpressionBuilder + .makeCast( + ConverterUtils.getTypeNode(sparkType, nullable = false), + ExpressionBuilder.makeSelection(colIdx, adjustedOrders(idx)), + SQLConf.get.ansiEnabled)) + } else { + // Velox and Spark have the same type + expressionNodes.add(ExpressionBuilder.makeSelection(colIdx, adjustedOrders(idx))) + } + } colIdx += 1 case _ => expressionNodes.add(ExpressionBuilder.makeSelection(colIdx)) @@ -225,78 +172,6 @@ case class HashAggregateExecTransformer( } } - /** - * Return the intermediate type node of a partial aggregation in Velox. - * @param aggregateFunction - * The aggregation function. - * @return - * The type of partial outputs. - */ - private def getIntermediateTypeNode(aggregateFunction: AggregateFunction): TypeNode = { - val structTypeNodes = new util.ArrayList[TypeNode]() - aggregateFunction match { - case avg: Average => - structTypeNodes.add( - ConverterUtils.getTypeNode(GlutenDecimalUtil.getAvgSumDataType(avg), nullable = true)) - structTypeNodes.add(ConverterUtils.getTypeNode(LongType, nullable = true)) - case first: First => - structTypeNodes.add(ConverterUtils.getTypeNode(first.dataType, nullable = true)) - structTypeNodes.add(ConverterUtils.getTypeNode(BooleanType, nullable = true)) - case last: Last => - structTypeNodes.add(ConverterUtils.getTypeNode(last.dataType, nullable = true)) - structTypeNodes.add(ConverterUtils.getTypeNode(BooleanType, nullable = true)) - case _: StddevSamp | _: StddevPop | _: VarianceSamp | _: VariancePop => - // Use struct type to represent Velox Row(BIGINT, DOUBLE, DOUBLE). - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxThreeIntermediateTypes.head, nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxThreeIntermediateTypes(1), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxThreeIntermediateTypes(2), nullable = false)) - case _: Corr => - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes.head, nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes(1), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes(2), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes(3), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes(4), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxSixIntermediateTypes(5), nullable = false)) - case _: CovPopulation | _: CovSample => - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxFourIntermediateTypes.head, nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxFourIntermediateTypes(1), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxFourIntermediateTypes(2), nullable = false)) - structTypeNodes.add( - ConverterUtils - .getTypeNode(veloxFourIntermediateTypes(3), nullable = false)) - case sum: Sum if sum.dataType.isInstanceOf[DecimalType] => - structTypeNodes.add(ConverterUtils.getTypeNode(sum.dataType, nullable = true)) - structTypeNodes.add(ConverterUtils.getTypeNode(BooleanType, nullable = false)) - case other => - throw new UnsupportedOperationException(s"$other is not supported.") - } - TypeBuilder.makeStruct(false, structTypeNodes) - } - override protected def modeToKeyWord(aggregateMode: AggregateMode): String = { super.modeToKeyWord(if (mixedPartialAndMerge) Partial else aggregateMode) } @@ -305,9 +180,9 @@ case class HashAggregateExecTransformer( override protected def addFunctionNode( args: java.lang.Object, aggregateFunction: AggregateFunction, - childrenNodeList: java.util.ArrayList[ExpressionNode], + childrenNodeList: JList[ExpressionNode], aggregateMode: AggregateMode, - aggregateNodeList: java.util.ArrayList[AggregateFunctionNode]): Unit = { + aggregateNodeList: JList[AggregateFunctionNode]): Unit = { // This is a special handling for PartialMerge in the execution of distinct. // Use Partial phase instead for this aggregation. val modeKeyWord = modeToKeyWord(aggregateMode) @@ -319,7 +194,8 @@ case class HashAggregateExecTransformer( VeloxAggregateFunctionsBuilder.create(args, aggregateFunction), childrenNodeList, modeKeyWord, - getIntermediateTypeNode(aggregateFunction)) + VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction) + ) aggregateNodeList.add(partialNode) case PartialMerge => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( @@ -327,7 +203,7 @@ case class HashAggregateExecTransformer( .create(args, aggregateFunction, mixedPartialAndMerge), childrenNodeList, modeKeyWord, - getIntermediateTypeNode(aggregateFunction) + VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction) ) aggregateNodeList.add(aggFunctionNode) case Final => @@ -372,7 +248,7 @@ case class HashAggregateExecTransformer( case sum: Sum if sum.dataType.isInstanceOf[DecimalType] => generateMergeCompanionNode() case _: Average | _: StddevSamp | _: StddevPop | _: VarianceSamp | _: VariancePop | _: Corr | - _: CovPopulation | _: CovSample | _: First | _: Last => + _: CovPopulation | _: CovSample | _: First | _: Last | _: MaxMinBy => generateMergeCompanionNode() case _ => val aggFunctionNode = ExpressionBuilder.makeAggregateFunction( @@ -392,8 +268,8 @@ case class HashAggregateExecTransformer( * Return the output types after partial aggregation through Velox. * @return */ - def getPartialAggOutTypes: java.util.ArrayList[TypeNode] = { - val typeNodeList = new java.util.ArrayList[TypeNode]() + def getPartialAggOutTypes: JList[TypeNode] = { + val typeNodeList = new JArrayList[TypeNode]() groupingExpressions.foreach( expression => { typeNodeList.add(ConverterUtils.getTypeNode(expression.dataType, expression.nullable)) @@ -404,10 +280,10 @@ case class HashAggregateExecTransformer( val aggregateFunction = expression.aggregateFunction aggregateFunction match { case _: Average | _: First | _: Last | _: StddevSamp | _: StddevPop | _: VarianceSamp | - _: VariancePop | _: Corr | _: CovPopulation | _: CovSample => + _: VariancePop | _: Corr | _: CovPopulation | _: CovSample | _: MaxMinBy => expression.mode match { case Partial | PartialMerge => - typeNodeList.add(getIntermediateTypeNode(aggregateFunction)) + typeNodeList.add(VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction)) case Final => typeNodeList.add( ConverterUtils @@ -418,7 +294,7 @@ case class HashAggregateExecTransformer( case sum: Sum if sum.dataType.isInstanceOf[DecimalType] => expression.mode match { case Partial | PartialMerge => - typeNodeList.add(getIntermediateTypeNode(aggregateFunction)) + typeNodeList.add(VeloxIntermediateData.getIntermediateTypeNode(aggregateFunction)) case Final => typeNodeList.add( ConverterUtils @@ -437,34 +313,32 @@ case class HashAggregateExecTransformer( // Return whether the outputs partial aggregation should be combined for Velox computing. // When the partial outputs are multiple-column, row construct is needed. private def rowConstructNeeded: Boolean = { - for (aggregateExpression <- aggregateExpressions) { - aggregateExpression.mode match { - case PartialMerge | Final => - if (aggregateExpression.aggregateFunction.inputAggBufferAttributes.size > 1) { - return true - } - case _ => - } + aggregateExpressions.exists { + aggExpr => + aggExpr.mode match { + case PartialMerge | Final => + aggExpr.aggregateFunction.inputAggBufferAttributes.size > 1 + case _ => false + } } - false } // Return a scalar function node representing row construct function in Velox. private def getRowConstructNode( args: java.lang.Object, - childNodes: util.ArrayList[ExpressionNode], - rowConstructAttributes: Seq[Attribute]): ScalarFunctionNode = { - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + childNodes: JList[ExpressionNode], + rowConstructAttributes: Seq[Attribute], + withNull: Boolean = true): ScalarFunctionNode = { + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionName = ConverterUtils.makeFuncName( - "row_constructor_with_null", + if (withNull) "row_constructor_with_null" else "row_constructor", rowConstructAttributes.map(attr => attr.dataType)) val functionId = ExpressionBuilder.newScalarFunction(functionMap, functionName) // Use struct type to represent Velox RowType. - val structTypeNodes = new util.ArrayList[TypeNode]( - rowConstructAttributes - .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - .asJava) + val structTypeNodes = rowConstructAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava ExpressionBuilder.makeScalarFunction( functionId, @@ -483,7 +357,7 @@ case class HashAggregateExecTransformer( validation: Boolean): RelNode = { val args = context.registeredFunction // Create a projection for row construct. - val exprNodes = new util.ArrayList[ExpressionNode]() + val exprNodes = new JArrayList[ExpressionNode]() groupingExpressions.foreach( expr => { exprNodes.add( @@ -497,7 +371,7 @@ case class HashAggregateExecTransformer( val aggregateFunction = aggregateExpression.aggregateFunction aggregateFunction match { case _ if mixedPartialAndMerge && aggregateExpression.mode == Partial => - val childNodes = new util.ArrayList[ExpressionNode]( + val childNodes = new JArrayList[ExpressionNode]( aggregateFunction.children .map( attr => { @@ -507,42 +381,44 @@ case class HashAggregateExecTransformer( }) .asJava) exprNodes.addAll(childNodes) - case Average(_, _) => + case avg: Average => aggregateExpression.mode match { case PartialMerge | Final => assert( functionInputAttributes.size == 2, s"${aggregateExpression.mode.toString} of Average expects two input attributes.") // Use a Velox function to combine the intermediate columns into struct. - val childNodes = new util.ArrayList[ExpressionNode]( + val childNodes = functionInputAttributes.toList .map( - attr => { - ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - .doTransform(args) - }) - .asJava) - exprNodes.add(getRowConstructNode(args, childNodes, functionInputAttributes)) + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args)) + .asJava + exprNodes.add( + getRowConstructNode( + args, + childNodes, + functionInputAttributes, + withNull = !avg.dataType.isInstanceOf[DecimalType])) case other => throw new UnsupportedOperationException(s"$other is not supported.") } - case _: First | _: Last => + case _: First | _: Last | _: MaxMinBy => aggregateExpression.mode match { case PartialMerge | Final => assert( functionInputAttributes.size == 2, - s"${aggregateExpression.mode.toString} of First/Last expects two input attributes.") + s"${aggregateExpression.mode.toString} of " + + s"${aggregateFunction.getClass.toString} expects two input attributes.") // Use a Velox function to combine the intermediate columns into struct. - val childNodes = new util.ArrayList[ExpressionNode]( - functionInputAttributes.toList - .map( - attr => { - ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - .doTransform(args) - }) - .asJava) + val childNodes = functionInputAttributes.toList + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args) + ) + .asJava exprNodes.add(getRowConstructNode(args, childNodes, functionInputAttributes)) case other => throw new UnsupportedOperationException(s"$other is not supported.") @@ -558,31 +434,28 @@ case class HashAggregateExecTransformer( // Use a Velox function to combine the intermediate columns into struct. var index = 0 var newInputAttributes: Seq[Attribute] = Seq() - val childNodes = new util.ArrayList[ExpressionNode]( - functionInputAttributes.toList - .map( - attr => { - val aggExpr: ExpressionTransformer = ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - val aggNode = aggExpr.doTransform(args) - val expressionNode = if (index == 0) { - // Cast count from DoubleType into LongType to align with Velox semantics. - newInputAttributes = newInputAttributes :+ - attr.copy(attr.name, LongType, attr.nullable, attr.metadata)( - attr.exprId, - attr.qualifier) - ExpressionBuilder.makeCast( - ConverterUtils.getTypeNode(LongType, attr.nullable), - aggNode, - SQLConf.get.ansiEnabled) - } else { - newInputAttributes = newInputAttributes :+ attr - aggNode - } - index += 1 - expressionNode - }) - .asJava) + val childNodes = functionInputAttributes.toList.map { + attr => + val aggExpr: ExpressionTransformer = ExpressionConverter + .replaceWithExpressionTransformer(attr, originalInputAttributes) + val aggNode = aggExpr.doTransform(args) + val expressionNode = if (index == 0) { + // Cast count from DoubleType into LongType to align with Velox semantics. + newInputAttributes = newInputAttributes :+ + attr.copy(attr.name, LongType, attr.nullable, attr.metadata)( + attr.exprId, + attr.qualifier) + ExpressionBuilder.makeCast( + ConverterUtils.getTypeNode(LongType, attr.nullable), + aggNode, + SQLConf.get.ansiEnabled) + } else { + newInputAttributes = newInputAttributes :+ attr + aggNode + } + index += 1 + expressionNode + }.asJava exprNodes.add(getRowConstructNode(args, childNodes, newInputAttributes)) case other => throw new UnsupportedOperationException(s"$other is not supported.") @@ -596,12 +469,12 @@ case class HashAggregateExecTransformer( // Use a Velox function to combine the intermediate columns into struct. var index = 0 var newInputAttributes: Seq[Attribute] = Seq() - val childNodes = new util.ArrayList[ExpressionNode]() + val childNodes = new JArrayList[ExpressionNode]() // Velox's Corr order is [ck, n, xMk, yMk, xAvg, yAvg] // Spark's Corr order is [n, xAvg, yAvg, ck, xMk, yMk] val sparkCorrOutputAttr = aggregateFunction.inputAggBufferAttributes.map(_.name) val veloxInputOrder = - VeloxAggregateFunctionsBuilder.veloxCorrIntermediateDataOrder.map( + VeloxIntermediateData.veloxCorrIntermediateDataOrder.map( name => sparkCorrOutputAttr.indexOf(name)) for (order <- veloxInputOrder) { val attr = functionInputAttributes(order) @@ -639,12 +512,12 @@ case class HashAggregateExecTransformer( // Use a Velox function to combine the intermediate columns into struct. var index = 0 var newInputAttributes: Seq[Attribute] = Seq() - val childNodes = new util.ArrayList[ExpressionNode]() + val childNodes = new JArrayList[ExpressionNode]() // Velox's Covar order is [ck, n, xAvg, yAvg] // Spark's Covar order is [n, xAvg, yAvg, ck] val sparkCorrOutputAttr = aggregateFunction.inputAggBufferAttributes.map(_.name) val veloxInputOrder = - VeloxAggregateFunctionsBuilder.veloxCovarIntermediateDataOrder.map( + VeloxIntermediateData.veloxCovarIntermediateDataOrder.map( name => sparkCorrOutputAttr.indexOf(name)) for (order <- veloxInputOrder) { val attr = functionInputAttributes(order) @@ -679,16 +552,15 @@ case class HashAggregateExecTransformer( functionInputAttributes.size == 2, "Final stage of Average expects two input attributes.") // Use a Velox function to combine the intermediate columns into struct. - val childNodes = new util.ArrayList[ExpressionNode]( - functionInputAttributes.toList - .map( - attr => { - ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - .doTransform(args) - }) - .asJava) - exprNodes.add(getRowConstructNode(args, childNodes, functionInputAttributes)) + val childNodes = functionInputAttributes.toList + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args) + ) + .asJava + exprNodes.add( + getRowConstructNode(args, childNodes, functionInputAttributes, withNull = false)) case other => throw new UnsupportedOperationException(s"$other is not supported.") } @@ -696,15 +568,13 @@ case class HashAggregateExecTransformer( if (functionInputAttributes.size != 1) { throw new UnsupportedOperationException("Only one input attribute is expected.") } - val childNodes = new util.ArrayList[ExpressionNode]( - functionInputAttributes.toList - .map( - attr => { - ExpressionConverter - .replaceWithExpressionTransformer(attr, originalInputAttributes) - .doTransform(args) - }) - .asJava) + val childNodes = functionInputAttributes.toList + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args) + ) + .asJava exprNodes.addAll(childNodes) } } @@ -715,10 +585,9 @@ case class HashAggregateExecTransformer( RelBuilder.makeProjectRel(inputRel, exprNodes, context, operatorId, emitStartIndex) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeProjectRel( @@ -731,7 +600,7 @@ case class HashAggregateExecTransformer( } // Create aggregation rel. - val groupingList = new util.ArrayList[ExpressionNode]() + val groupingList = new JArrayList[ExpressionNode]() var colIdx = 0 groupingExpressions.foreach( _ => { @@ -739,8 +608,8 @@ case class HashAggregateExecTransformer( colIdx += 1 }) - val aggFilterList = new util.ArrayList[ExpressionNode]() - val aggregateFunctionList = new util.ArrayList[AggregateFunctionNode]() + val aggFilterList = new JArrayList[ExpressionNode]() + val aggregateFunctionList = new JArrayList[AggregateFunctionNode]() aggregateExpressions.foreach( aggExpr => { if (aggExpr.filter.isDefined) { @@ -751,10 +620,10 @@ case class HashAggregateExecTransformer( } val aggregateFunc = aggExpr.aggregateFunction - val childrenNodes = new util.ArrayList[ExpressionNode]() + val childrenNodes = new JArrayList[ExpressionNode]() aggregateFunc match { - case _: Average | _: First | _: Last | _: StddevSamp | _: StddevPop | - _: VarianceSamp | _: VariancePop | _: Corr | _: CovPopulation | _: CovSample + case _: Average | _: First | _: Last | _: StddevSamp | _: StddevPop | _: VarianceSamp | + _: VariancePop | _: Corr | _: CovPopulation | _: CovSample | _: MaxMinBy if aggExpr.mode == PartialMerge | aggExpr.mode == Final => // Only occupies one column due to intermediate results are combined // by previous projection. @@ -800,14 +669,8 @@ case class HashAggregateExecTransformer( * whether partial and partial-merge functions coexist. */ def mixedPartialAndMerge: Boolean = { - val partialMergeExists = aggregateExpressions.exists( - expression => { - expression.mode == PartialMerge - }) - val partialExists = aggregateExpressions.exists( - expression => { - expression.mode == Partial - }) + val partialMergeExists = aggregateExpressions.exists(_.mode == PartialMerge) + val partialExists = aggregateExpressions.exists(_.mode == Partial) partialMergeExists && partialExists } @@ -874,60 +737,6 @@ case class HashAggregateExecTransformer( /** An aggregation function builder specifically used by Velox backend. */ object VeloxAggregateFunctionsBuilder { - val veloxCorrIntermediateDataOrder: Seq[String] = Seq("ck", "n", "xMk", "yMk", "xAvg", "yAvg") - val veloxCovarIntermediateDataOrder: Seq[String] = Seq("ck", "n", "xAvg", "yAvg") - - val veloxThreeIntermediateTypes: Seq[DataType] = Seq(LongType, DoubleType, DoubleType) - val veloxFourIntermediateTypes: Seq[DataType] = Seq(DoubleType, LongType, DoubleType, DoubleType) - val veloxSixIntermediateTypes: Seq[DataType] = - Seq(DoubleType, LongType, DoubleType, DoubleType, DoubleType, DoubleType) - - /** - * Get the compatible input types for a Velox aggregate function. - * @param aggregateFunc: - * the input aggreagate function. - * @param forMergeCompanion: - * whether this is a special case to solve mixed aggregation phases. - * @return - * the input types of a Velox aggregate function. - */ - private def getInputTypes( - aggregateFunc: AggregateFunction, - forMergeCompanion: Boolean): Seq[DataType] = { - if (!forMergeCompanion) { - return aggregateFunc.children.map(child => child.dataType) - } - if (aggregateFunc.aggBufferAttributes.size == veloxThreeIntermediateTypes.size) { - return Seq( - StructType( - veloxThreeIntermediateTypes - .map(intermediateType => StructField("", intermediateType)) - .toArray)) - } - if (aggregateFunc.aggBufferAttributes.size == veloxFourIntermediateTypes.size) { - return Seq( - StructType( - veloxFourIntermediateTypes - .map(intermediateType => StructField("", intermediateType)) - .toArray)) - } - if (aggregateFunc.aggBufferAttributes.size == veloxSixIntermediateTypes.size) { - return Seq( - StructType( - veloxSixIntermediateTypes - .map(intermediateType => StructField("", intermediateType)) - .toArray)) - } - if (aggregateFunc.aggBufferAttributes.size > 1) { - return Seq( - StructType( - aggregateFunc.aggBufferAttributes - .map(attribute => StructField("", attribute.dataType)) - .toArray)) - } - aggregateFunc.aggBufferAttributes.map(child => child.dataType) - } - /** * Create an scalar function for the input aggregate function. * @param args: @@ -942,7 +751,7 @@ object VeloxAggregateFunctionsBuilder { args: java.lang.Object, aggregateFunc: AggregateFunction, forMergeCompanion: Boolean = false): Long = { - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] var sigName = ExpressionMappings.expressionsMap.get(aggregateFunc.getClass) if (sigName.isEmpty) { @@ -964,7 +773,7 @@ object VeloxAggregateFunctionsBuilder { functionMap, ConverterUtils.makeFuncName( substraitAggFuncName, - getInputTypes(aggregateFunc, forMergeCompanion), + VeloxIntermediateData.getInputTypes(aggregateFunc, forMergeCompanion), FunctionConfig.REQ)) } } diff --git a/backends-velox/src/main/scala/io/glutenproject/execution/RowToVeloxColumnarExec.scala b/backends-velox/src/main/scala/io/glutenproject/execution/RowToVeloxColumnarExec.scala index bde36003b9f4..d898d9f18b53 100644 --- a/backends-velox/src/main/scala/io/glutenproject/execution/RowToVeloxColumnarExec.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/RowToVeloxColumnarExec.scala @@ -18,10 +18,10 @@ package io.glutenproject.execution import io.glutenproject.backendsapi.velox.ValidatorApiImpl import io.glutenproject.columnarbatch.ColumnarBatches -import io.glutenproject.exec.ExecutionCtxs +import io.glutenproject.exec.Runtimes import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers -import io.glutenproject.utils.ArrowAbiUtil +import io.glutenproject.utils.{ArrowAbiUtil, Iterators} import io.glutenproject.vectorized._ import org.apache.spark.rdd.RDD @@ -93,7 +93,6 @@ object RowToVeloxColumnarExec { val jniWrapper = NativeRowToColumnarJniWrapper.create() val allocator = ArrowBufferAllocators.contextInstance() val cSchema = ArrowSchema.allocateNew(allocator) - var closed = false val r2cHandle = try { ArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema) @@ -106,22 +105,15 @@ object RowToVeloxColumnarExec { cSchema.close() } - TaskResources.addRecycler(s"RowToColumnar_$r2cHandle", 100) { - if (!closed) { - jniWrapper.close(r2cHandle) - closed = true - } - } - val res: Iterator[ColumnarBatch] = new Iterator[ColumnarBatch] { + var finished = false override def hasNext: Boolean = { - val itHasNext = it.hasNext - if (!itHasNext && !closed) { - jniWrapper.close(r2cHandle) - closed = true + if (finished) { + false + } else { + it.hasNext } - itHasNext } def nativeConvert(row: UnsafeRow): ColumnarBatch = { @@ -154,31 +146,36 @@ object RowToVeloxColumnarExec { rowLength += sizeInBytes.toLong rowCount += 1 - while (rowCount < columnBatchSize && it.hasNext) { - val row = it.next() - val unsafeRow = convertToUnsafeRow(row) - val sizeInBytes = unsafeRow.getSizeInBytes - if ((offset + sizeInBytes) > arrowBuf.capacity()) { - val tmpBuf = allocator.buffer(((offset + sizeInBytes) * 2).toLong) - tmpBuf.setBytes(0, arrowBuf, 0, offset) - arrowBuf.close() - arrowBuf = tmpBuf + while (rowCount < columnBatchSize && !finished) { + val iterHasNext = it.hasNext + if (!iterHasNext) { + finished = true + } else { + val row = it.next() + val unsafeRow = convertToUnsafeRow(row) + val sizeInBytes = unsafeRow.getSizeInBytes + if ((offset + sizeInBytes) > arrowBuf.capacity()) { + val tmpBuf = allocator.buffer(((offset + sizeInBytes) * 2).toLong) + tmpBuf.setBytes(0, arrowBuf, 0, offset) + arrowBuf.close() + arrowBuf = tmpBuf + } + Platform.copyMemory( + unsafeRow.getBaseObject, + unsafeRow.getBaseOffset, + null, + arrowBuf.memoryAddress() + offset, + sizeInBytes) + offset += sizeInBytes + rowLength += sizeInBytes.toLong + rowCount += 1 } - Platform.copyMemory( - unsafeRow.getBaseObject, - unsafeRow.getBaseOffset, - null, - arrowBuf.memoryAddress() + offset, - sizeInBytes) - offset += sizeInBytes - rowLength += sizeInBytes.toLong - rowCount += 1 } numInputRows += rowCount try { val handle = jniWrapper .nativeConvertRowToColumnar(r2cHandle, rowLength.toArray, arrowBuf.memoryAddress()) - ColumnarBatches.create(ExecutionCtxs.contextInstance(), handle) + ColumnarBatches.create(Runtimes.contextInstance(), handle) } finally { arrowBuf.close() arrowBuf = null @@ -205,6 +202,12 @@ object RowToVeloxColumnarExec { cb } } - new CloseableColumnBatchIterator(res) + Iterators + .wrap(res) + .recycleIterator { + jniWrapper.close(r2cHandle) + } + .recyclePayload(_.close()) + .create() } } diff --git a/backends-velox/src/main/scala/io/glutenproject/execution/VeloxColumnarToRowExec.scala b/backends-velox/src/main/scala/io/glutenproject/execution/VeloxColumnarToRowExec.scala index e0659a9c67a5..3428e8b7e24a 100644 --- a/backends-velox/src/main/scala/io/glutenproject/execution/VeloxColumnarToRowExec.scala +++ b/backends-velox/src/main/scala/io/glutenproject/execution/VeloxColumnarToRowExec.scala @@ -19,6 +19,7 @@ package io.glutenproject.execution import io.glutenproject.columnarbatch.ColumnarBatches import io.glutenproject.extension.ValidationResult import io.glutenproject.memory.nmm.NativeMemoryManagers +import io.glutenproject.utils.Iterators import io.glutenproject.vectorized.NativeColumnarToRowJniWrapper import org.apache.spark.rdd.RDD @@ -28,7 +29,6 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.TaskResources import scala.collection.JavaConverters._ @@ -98,26 +98,13 @@ object VeloxColumnarToRowExec { // TODO:: pass the jni jniWrapper and arrowSchema and serializeSchema method by broadcast val jniWrapper = NativeColumnarToRowJniWrapper.create() - var closed = false val c2rId = jniWrapper.nativeColumnarToRowInit( NativeMemoryManagers.contextInstance("ColumnarToRow").getNativeInstanceHandle) - TaskResources.addRecycler(s"ColumnarToRow_$c2rId", 100) { - if (!closed) { - jniWrapper.nativeClose(c2rId) - closed = true - } - } - val res: Iterator[Iterator[InternalRow]] = new Iterator[Iterator[InternalRow]] { override def hasNext: Boolean = { - val hasNext = batches.hasNext - if (!hasNext && !closed) { - jniWrapper.nativeClose(c2rId) - closed = true - } - hasNext + batches.hasNext } override def next(): Iterator[InternalRow] = { @@ -170,6 +157,13 @@ object VeloxColumnarToRowExec { } } } - res.flatten + Iterators + .wrap(res.flatten) + .protectInvocationFlow() // Spark may call `hasNext()` again after a false output which + // is not allowed by Gluten iterators. E.g. GroupedIterator#fetchNextGroupIterator + .recycleIterator { + jniWrapper.nativeClose(c2rId) + } + .create() } } diff --git a/backends-velox/src/main/scala/io/glutenproject/expression/ExpressionTransformer.scala b/backends-velox/src/main/scala/io/glutenproject/expression/ExpressionTransformer.scala index cd98b94549bb..34db622835f0 100644 --- a/backends-velox/src/main/scala/io/glutenproject/expression/ExpressionTransformer.scala +++ b/backends-velox/src/main/scala/io/glutenproject/expression/ExpressionTransformer.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.types.{IntegerType, LongType} import com.google.common.collect.Lists +import java.lang.{Integer => JInteger, Long => JLong} +import java.util.{ArrayList => JArrayList, HashMap => JHashMap} + import scala.language.existentials case class VeloxAliasTransformer( @@ -49,7 +52,7 @@ case class VeloxNamedStructTransformer( child => expressionNodes.add( replaceWithExpressionTransformer(child, attributeSeq).doTransform(args))) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionName = ConverterUtils .makeFuncName(substraitExprName, Seq(original.dataType), FunctionConfig.OPT) val functionId = ExpressionBuilder.newScalarFunction(functionMap, functionName) @@ -71,7 +74,7 @@ case class VeloxGetStructFieldTransformer( node.getFieldLiteral(ordinal) case node: SelectionNode => // Append the nested index to selection node. - node.addNestedChildIdx(java.lang.Integer.valueOf(ordinal)) + node.addNestedChildIdx(JInteger.valueOf(ordinal)) case other => throw new UnsupportedOperationException(s"$other is not supported.") } @@ -94,7 +97,7 @@ case class VeloxHashExpressionTransformer( case HiveHash(_) => (ExpressionBuilder.makeIntLiteral(0), IntegerType) } - val nodes = new java.util.ArrayList[ExpressionNode]() + val nodes = new JArrayList[ExpressionNode]() // Seed as the first argument nodes.add(seedNode) exps.foreach( @@ -102,7 +105,7 @@ case class VeloxHashExpressionTransformer( nodes.add(expression.doTransform(args)) }) val childrenTypes = seedType +: original.children.map(child => child.dataType) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionName = ConverterUtils.makeFuncName(substraitExprName, childrenTypes, FunctionConfig.OPT) val functionId = ExpressionBuilder.newScalarFunction(functionMap, functionName) diff --git a/backends-velox/src/main/scala/io/glutenproject/utils/VeloxIntermediateData.scala b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxIntermediateData.scala new file mode 100644 index 000000000000..7c4d5ecc00a7 --- /dev/null +++ b/backends-velox/src/main/scala/io/glutenproject/utils/VeloxIntermediateData.scala @@ -0,0 +1,130 @@ +/* + * 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 io.glutenproject.utils + +import io.glutenproject.expression.ConverterUtils +import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} + +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.types._ + +import scala.collection.JavaConverters._ + +object VeloxIntermediateData { + // Agg functions with inconsistent ordering of intermediate data between Velox and Spark. + // Corr + val veloxCorrIntermediateDataOrder: Seq[String] = Seq("ck", "n", "xMk", "yMk", "xAvg", "yAvg") + // CovPopulation, CovSample + val veloxCovarIntermediateDataOrder: Seq[String] = Seq("ck", "n", "xAvg", "yAvg") + + // Agg functions with inconsistent types of intermediate data between Velox and Spark. + // StddevSamp, StddevPop, VarianceSamp, VariancePop + val veloxVarianceIntermediateTypes: Seq[DataType] = Seq(LongType, DoubleType, DoubleType) + // CovPopulation, CovSample + val veloxCovarIntermediateTypes: Seq[DataType] = Seq(DoubleType, LongType, DoubleType, DoubleType) + // Corr + val veloxCorrIntermediateTypes: Seq[DataType] = + Seq(DoubleType, LongType, DoubleType, DoubleType, DoubleType, DoubleType) + + /** + * Return the intermediate columns order of Velox aggregation functions, with special matching + * required for some aggregation functions where the intermediate columns order are inconsistent + * with Spark. + * @param aggFunc + * Spark aggregation function + * @return + * the intermediate columns order of Velox aggregation functions + */ + def veloxIntermediateDataOrder(aggFunc: AggregateFunction): Seq[String] = { + aggFunc match { + case _: Corr => + veloxCorrIntermediateDataOrder + case _: CovPopulation | _: CovSample => + veloxCovarIntermediateDataOrder + case _ => + aggFunc.aggBufferAttributes.map(_.name) + } + } + + /** + * Get the compatible input types for a Velox aggregate function. + * + * @param aggregateFunc + * The input aggregate function. + * @param forMergeCompanion + * Whether this is a special case to solve mixed aggregation phases. + * @return + * The input types of a Velox aggregate function. + */ + def getInputTypes(aggregateFunc: AggregateFunction, forMergeCompanion: Boolean): Seq[DataType] = { + if (!forMergeCompanion) { + return aggregateFunc.children.map(_.dataType) + } + aggregateFunc match { + case _ @Type(veloxDataTypes: Seq[DataType]) => + Seq(StructType(veloxDataTypes.map(StructField("", _)).toArray)) + case _ => + // Not use StructType for single column agg intermediate data + aggregateFunc.aggBufferAttributes.map(_.dataType) + } + } + + /** + * Return the intermediate type node of a partial aggregation in Velox. + * + * @param aggFunc + * Spark aggregation function. + * @return + * The type of partial outputs. + */ + def getIntermediateTypeNode(aggFunc: AggregateFunction): TypeNode = { + val structTypeNodes = + aggFunc match { + case _ @Type(dataTypes: Seq[DataType]) => + dataTypes.map(ConverterUtils.getTypeNode(_, nullable = false)) + case _ => + throw new UnsupportedOperationException("Can not get velox intermediate types.") + } + TypeBuilder.makeStruct(false, structTypeNodes.asJava) + } + + object Type { + + /** + * Return the intermediate types of Velox agg functions, with special matching required for some + * aggregation functions where the intermediate results are inconsistent with Spark. Only return + * if the intermediate result has multiple columns. + * @param aggFunc + * Spark aggregation function + * @return + * the intermediate types of Velox aggregation functions. + */ + def unapply(aggFunc: AggregateFunction): Option[Seq[DataType]] = { + aggFunc match { + case _: Corr => + Some(veloxCorrIntermediateTypes) + case _: Covariance => + Some(veloxCovarIntermediateTypes) + case _: StddevSamp | _: StddevPop | _: VarianceSamp | _: VariancePop => + Some(veloxVarianceIntermediateTypes) + case _ if aggFunc.aggBufferAttributes.size > 1 => + Some(aggFunc.aggBufferAttributes.map(_.dataType)) + case _ => None + } + } + } +} 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 836bcd29fb3f..1012d8027957 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 @@ -20,12 +20,12 @@ import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.backendsapi.velox.ValidatorApiImpl import io.glutenproject.columnarbatch.ColumnarBatches -import io.glutenproject.exec.ExecutionCtxs +import io.glutenproject.exec.Runtimes import io.glutenproject.execution.{RowToVeloxColumnarExec, VeloxColumnarToRowExec} import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers -import io.glutenproject.utils.ArrowAbiUtil -import io.glutenproject.vectorized.{CloseableColumnBatchIterator, ColumnarBatchSerializerJniWrapper} +import io.glutenproject.utils.{ArrowAbiUtil, Iterators} +import io.glutenproject.vectorized.ColumnarBatchSerializerJniWrapper import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession @@ -38,7 +38,6 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.utils.SparkArrowUtil import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.TaskResources import org.apache.arrow.c.ArrowSchema @@ -226,7 +225,7 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe val requestedColumnIndices = selectedAttributes.map { a => cacheAttributes.map(_.exprId).indexOf(a.exprId) } - val shouldPruning = selectedAttributes.size != cacheAttributes.size + val shouldSelectAttributes = cacheAttributes != selectedAttributes val localSchema = toStructType(cacheAttributes) val timezoneId = SQLConf.get.sessionLocalTimeZone input.mapPartitions { @@ -244,33 +243,34 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with SQLConfHe nmm.getNativeInstanceHandle ) cSchema.close() - TaskResources.addRecycler( - s"ColumnarCachedBatchSerializer_convertCachedBatchToColumnarBatch_$deserializerHandle", - 50) { - ColumnarBatchSerializerJniWrapper.create().close(deserializerHandle) - } - new CloseableColumnBatchIterator(new Iterator[ColumnarBatch] { - override def hasNext: Boolean = it.hasNext + Iterators + .wrap(new Iterator[ColumnarBatch] { + override def hasNext: Boolean = it.hasNext - override def next(): ColumnarBatch = { - val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch] - val batchHandle = - ColumnarBatchSerializerJniWrapper - .create() - .deserialize(deserializerHandle, cachedBatch.bytes) - val batch = ColumnarBatches.create(ExecutionCtxs.contextInstance(), batchHandle) - if (shouldPruning) { - try { - ColumnarBatches.select(nmm, batch, requestedColumnIndices.toArray) - } finally { - batch.close() + override def next(): ColumnarBatch = { + val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch] + val batchHandle = + ColumnarBatchSerializerJniWrapper + .create() + .deserialize(deserializerHandle, cachedBatch.bytes) + val batch = ColumnarBatches.create(Runtimes.contextInstance(), batchHandle) + if (shouldSelectAttributes) { + try { + ColumnarBatches.select(nmm, batch, requestedColumnIndices.toArray) + } finally { + batch.close() + } + } else { + batch } - } else { - batch } + }) + .recycleIterator { + ColumnarBatchSerializerJniWrapper.create().close(deserializerHandle) } - }) + .recyclePayload(_.close()) + .create() } } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala index 3ff69a24737c..1aee3ed3394c 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/VeloxWriteQueue.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.execution.datasources import io.glutenproject.datasource.DatasourceJniWrapper -import io.glutenproject.vectorized.{CloseableColumnBatchIterator, ColumnarBatchInIterator} +import io.glutenproject.utils.Iterators +import io.glutenproject.vectorized.ColumnarBatchInIterator import org.apache.spark.TaskContext import org.apache.spark.sql.execution.datasources.VeloxWriteQueue.EOS_BATCH @@ -50,7 +51,8 @@ class VeloxWriteQueue( try { datasourceJniWrapper.write( dsHandle, - new ColumnarBatchInIterator(new CloseableColumnBatchIterator(scanner).asJava)) + new ColumnarBatchInIterator( + Iterators.wrap(scanner).recyclePayload(_.close()).create().asJava)) } catch { case e: Exception => writeException.set(e) diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxBlockStripes.java b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxBlockStripes.java index e1878ce32480..65ec48db9a36 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxBlockStripes.java +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxBlockStripes.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.velox; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtimes; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; import org.apache.spark.sql.execution.datasources.BlockStripe; @@ -53,7 +53,7 @@ public BlockStripe next() { return new BlockStripe() { @Override public ColumnarBatch getColumnarBatch() { - return ColumnarBatches.create(ExecutionCtxs.contextInstance(), blockAddresses[0]); + return ColumnarBatches.create(Runtimes.contextInstance(), blockAddresses[0]); } @Override 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 482a6a8495ce..3e83d292ad60 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 @@ -34,18 +34,26 @@ import org.apache.spark.util.TaskResources import com.google.common.base.Preconditions import org.apache.arrow.c.ArrowSchema -import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapreduce.TaskAttemptContext import java.io.IOException trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { def createOutputWriter( - path: String, + filePath: String, dataSchema: StructType, context: TaskAttemptContext, nativeConf: java.util.Map[String, String]): OutputWriter = { - val originPath = path + // Create the hdfs path if not existed. + val hdfsSchema = "hdfs://" + if (filePath.startsWith(hdfsSchema)) { + val fs = FileSystem.get(context.getConfiguration) + val hdfsPath = new Path(filePath) + if (!fs.exists(hdfsPath.getParent)) { + fs.mkdirs(hdfsPath.getParent) + } + } val arrowSchema = SparkArrowUtil.toArrowSchema(dataSchema, SQLConf.get.sessionLocalTimeZone) @@ -56,7 +64,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { try { ArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema) dsHandle = datasourceJniWrapper.nativeInitDatasource( - originPath, + filePath, cSchema.memoryAddress(), NativeMemoryManagers.contextInstance("VeloxWriter").getNativeInstanceHandle, nativeConf) @@ -74,7 +82,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { arrowSchema, allocator, datasourceJniWrapper, - originPath) + filePath) new OutputWriter { override def write(row: InternalRow): Unit = { @@ -91,7 +99,7 @@ trait VeloxFormatWriterInjects extends GlutenFormatWriterInjectsBase { // Do NOT add override keyword for compatibility on spark 3.1. def path(): String = { - originPath + filePath } } } diff --git a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxParquetWriterInjects.scala b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxParquetWriterInjects.scala index 0fdc225008b4..c758b62d066f 100644 --- a/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxParquetWriterInjects.scala +++ b/backends-velox/src/main/scala/org/apache/spark/sql/execution/datasources/velox/VeloxParquetWriterInjects.scala @@ -39,6 +39,9 @@ class VeloxParquetWriterInjects extends VeloxFormatWriterInjects { GlutenConfig.PARQUET_BLOCK_ROWS, GlutenConfig.getConf.columnarParquetWriteBlockRows.toString) sparkOptions.put(GlutenConfig.PARQUET_BLOCK_ROWS, blockRows) + options + .get(GlutenConfig.PARQUET_GZIP_WINDOW_SIZE) + .foreach(sparkOptions.put(GlutenConfig.PARQUET_GZIP_WINDOW_SIZE, _)) sparkOptions.asJava } diff --git a/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala b/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala index 2613f27010b0..4f35bf9ec2c7 100644 --- a/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala +++ b/backends-velox/src/test/scala/io/glutenproject/benchmarks/NativeBenchmarkPlanGenerator.scala @@ -16,6 +16,7 @@ */ package io.glutenproject.benchmarks +import io.glutenproject.GlutenConfig import io.glutenproject.execution.{VeloxWholeStageTransformerSuite, WholeStageTransformer} import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} @@ -50,8 +51,9 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { } test("Test plan json non-empty - AQE off") { - spark.sparkContext.setLogLevel("DEBUG") - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true") { val df = spark .sql(""" |select * from lineitem @@ -59,18 +61,19 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { val executedPlan = df.queryExecution.executedPlan val lastStageTransformer = executedPlan.find(_.isInstanceOf[WholeStageTransformer]) assert(lastStageTransformer.nonEmpty) - var planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].getPlanJson - assert(planJson.isEmpty) + var planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson + assert(planJson.nonEmpty) executedPlan.execute() - planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].getPlanJson + planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } spark.sparkContext.setLogLevel(logLevel) } test("Test plan json non-empty - AQE on") { - spark.sparkContext.setLogLevel("DEBUG") - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true") { val df = spark .sql(""" |select * from lineitem join orders on l_orderkey = o_orderkey @@ -82,7 +85,7 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { val finalPlan = executedPlan.asInstanceOf[AdaptiveSparkPlanExec].executedPlan val lastStageTransformer = finalPlan.find(_.isInstanceOf[WholeStageTransformer]) assert(lastStageTransformer.nonEmpty) - val planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].getPlanJson + val planJson = lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson assert(planJson.nonEmpty) } spark.sparkContext.setLogLevel(logLevel) @@ -90,10 +93,10 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { test("generate example", GenerateExample) { import testImplicits._ - spark.sparkContext.setLogLevel("DEBUG") withSQLConf( SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", - SQLConf.SHUFFLE_PARTITIONS.key -> "2" + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT.key -> "true" ) { val q4_lineitem = spark .sql(s""" @@ -141,7 +144,7 @@ class NativeBenchmarkPlanGenerator extends VeloxWholeStageTransformerSuite { val lastStageTransformer = finalPlan.find(_.isInstanceOf[WholeStageTransformer]) assert(lastStageTransformer.nonEmpty) val plan = - lastStageTransformer.get.asInstanceOf[WholeStageTransformer].getPlanJson.split('\n') + lastStageTransformer.get.asInstanceOf[WholeStageTransformer].substraitPlanJson.split('\n') val exampleJsonFile = Paths.get(generatedPlanDir, "example.json") Files.write(exampleJsonFile, plan.toList.asJava, StandardCharsets.UTF_8) diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala b/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala index afc29a87d9d6..7e8049e33a30 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/FallbackSuite.scala @@ -20,7 +20,7 @@ import io.glutenproject.GlutenConfig import org.apache.spark.SparkConf import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, ColumnarAQEShuffleReadExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanHelper, AQEShuffleReadExec} class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper { protected val rootPath: String = getClass.getResource("/").getPath @@ -116,10 +116,23 @@ class FallbackSuite extends VeloxWholeStageTransformerSuite with AdaptiveSparkPl ) { df => val aqeRead = find(df.queryExecution.executedPlan) { - case _: ColumnarAQEShuffleReadExec => true + case _: AQEShuffleReadExec => true case _ => false } assert(aqeRead.isDefined) } } + + test("Do not fallback eagerly with ColumnarToRowExec") { + withSQLConf(GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { + runQueryAndCompare("select count(*) from tmp1") { + df => + assert( + collect(df.queryExecution.executedPlan) { + case h: HashAggregateExecTransformer => h + }.size == 2, + df.queryExecution.executedPlan) + } + } + } } diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala b/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala index aceb8c109ca4..dbc1e1e471c1 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/TestOperator.scala @@ -18,13 +18,14 @@ package io.glutenproject.execution import org.apache.spark.SparkConf import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.RDDScanExec -import org.apache.spark.sql.functions.{avg, col} +import org.apache.spark.sql.execution.{GenerateExec, RDDScanExec} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.functions.{avg, col, udf} import org.apache.spark.sql.types.{DecimalType, StringType, StructField, StructType} import scala.collection.JavaConverters -class TestOperator extends VeloxWholeStageTransformerSuite { +class TestOperator extends VeloxWholeStageTransformerSuite with AdaptiveSparkPlanHelper { protected val rootPath: String = getClass.getResource("/").getPath override protected val backend: String = "velox" @@ -193,55 +194,59 @@ class TestOperator extends VeloxWholeStageTransformerSuite { } test("window expression") { - runQueryAndCompare( - "select row_number() over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select rank() over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select dense_rank() over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select percent_rank() over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select cume_dist() over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select l_suppkey, l_orderkey, nth_value(l_orderkey, 2) over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { - df => - { - assert( - getExecutedPlan(df).count( - plan => { - plan.isInstanceOf[WindowExecTransformer] - }) > 0) + Seq("sort", "streaming").foreach { + windowType => + withSQLConf("spark.gluten.sql.columnar.backend.velox.window.type" -> windowType.toString) { + runQueryAndCompare( + "select row_number() over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select rank() over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select dense_rank() over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select percent_rank() over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select cume_dist() over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select l_suppkey, l_orderkey, nth_value(l_orderkey, 2) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { + df => + { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[WindowExecTransformer] + }) > 0) + } + } + + runQueryAndCompare( + "select sum(l_partkey + 1) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem") { _ => } + + runQueryAndCompare( + "select max(l_partkey) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select min(l_partkey) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } + + runQueryAndCompare( + "select avg(l_partkey) over" + + " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } } } - - runQueryAndCompare( - "select sum(l_partkey + 1) over" + - " (partition by l_suppkey order by l_orderkey) from lineitem") { _ => } - - runQueryAndCompare( - "select max(l_partkey) over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select min(l_partkey) over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - - runQueryAndCompare( - "select avg(l_partkey) over" + - " (partition by l_suppkey order by l_orderkey) from lineitem ") { _ => } - } test("chr function") { @@ -458,7 +463,10 @@ class TestOperator extends VeloxWholeStageTransformerSuite { .select($"DecimalCol".cast(DecimalType(38, 33))) .select(col("DecimalCol")) .agg(avg($"DecimalCol")) - assert(result.collect()(0).get(0).toString.equals("0.0345678900000000000000000000000000000")) + // Double precision loss: + // https://github.com/facebookincubator/velox/pull/6051#issuecomment-1731028215. + // assert(result.collect()(0).get(0).toString.equals("0.0345678900000000000000000000000000000")) + assert((result.collect()(0).get(0).toString.toDouble - d).abs < 0.00000000001) checkOperatorMatch[HashAggregateExecTransformer](result) } @@ -542,6 +550,57 @@ class TestOperator extends VeloxWholeStageTransformerSuite { runQueryAndCompare("SELECT c1, explode(array(c2)) FROM t") { checkOperatorMatch[GenerateExecTransformer] } + + runQueryAndCompare("SELECT c1, explode(c3) FROM (SELECT c1, array(c2) as c3 FROM t)") { + checkOperatorMatch[GenerateExecTransformer] + } + } + } + + test("Validation should fail if unsupported expression is used for Generate.") { + withTable("t") { + spark + .range(10) + .selectExpr("id as c1", "id as c2") + .write + .format("parquet") + .saveAsTable("t") + + // Add a simple UDF to generate the unsupported case + val intToArrayFunc = udf((s: Int) => Array(s)) + spark.udf.register("intToArray", intToArrayFunc) + + // Testing unsupported case + runQueryAndCompare("SELECT explode(intToArray(c1)) from t;") { + df => + { + getExecutedPlan(df).exists(plan => plan.find(_.isInstanceOf[GenerateExec]).isDefined) + } + } + + // Testing unsupported case in case when + runQueryAndCompare(""" + |SELECT explode(case when size(intToArray(c1)) > 0 + |then array(c1) else array(c2) end) from t; + |""".stripMargin) { + df => + { + getExecutedPlan(df).exists(plan => plan.find(_.isInstanceOf[GenerateExec]).isDefined) + } + } + } + } + + test("Support get native plan tree string") { + runQueryAndCompare("select l_partkey + 1, count(*) from lineitem group by l_partkey + 1") { + df => + val wholeStageTransformers = collect(df.queryExecution.executedPlan) { + case w: WholeStageTransformer => w + } + val nativePlanString = wholeStageTransformers.head.nativePlanString() + assert(nativePlanString.contains("Aggregation[FINAL")) + assert(nativePlanString.contains("Aggregation[PARTIAL")) + assert(nativePlanString.contains("TableScan")) } } } diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala index 2dcce7d62a50..ec7f3337e251 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxAggregateFunctionsSuite.scala @@ -419,6 +419,48 @@ class VeloxAggregateFunctionsSuite extends VeloxWholeStageTransformerSuite { } } + test("max_by") { + runQueryAndCompare(s""" + |select max_by(l_linenumber, l_comment) from lineitem; + |""".stripMargin) { + checkOperatorMatch[HashAggregateExecTransformer] + } + runQueryAndCompare(s""" + |select max_by(distinct l_linenumber, l_comment) + |from lineitem + |""".stripMargin) { + df => + { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[HashAggregateExecTransformer] + }) == 4) + } + } + } + + test("min_by") { + runQueryAndCompare(s""" + |select min_by(l_linenumber, l_comment) from lineitem; + |""".stripMargin) { + checkOperatorMatch[HashAggregateExecTransformer] + } + runQueryAndCompare(s""" + |select min_by(distinct l_linenumber, l_comment) + |from lineitem + |""".stripMargin) { + df => + { + assert( + getExecutedPlan(df).count( + plan => { + plan.isInstanceOf[HashAggregateExecTransformer] + }) == 4) + } + } + } + test("distinct functions") { runQueryAndCompare("SELECT sum(DISTINCT l_partkey), count(*) FROM lineitem") { df => diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala index 0de9733d47a4..889f3e8d66a7 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxColumnarCacheSuite.scala @@ -17,6 +17,7 @@ package io.glutenproject.execution import io.glutenproject.GlutenConfig +import io.glutenproject.extension.InMemoryTableScanHelper import org.apache.spark.SparkConf import org.apache.spark.sql.Row @@ -39,6 +40,7 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt super.sparkConf .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.sql.shuffle.partitions", "3") + .set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") } private def checkColumnarTableCache(plan: SparkPlan): Unit = { @@ -124,6 +126,26 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt } } + test("no ColumnarToRow for table cache") { + val cached = spark.table("lineitem").cache() + withSQLConf(GlutenConfig.COLUMNAR_HASHAGG_ENABLED.key -> "false") { + try { + val df = spark.sql("SELECT COUNT(*) FROM lineitem") + checkAnswer(df, Row(60175)) + assert( + find(df.queryExecution.executedPlan) { + case VeloxColumnarToRowExec(child: SparkPlan) + if InMemoryTableScanHelper.isGlutenTableCache(child) => + true + case _ => false + }.isEmpty + ) + } finally { + cached.unpersist() + } + } + } + test("Columnar table cache should compatible with TableCacheQueryStage") { withSQLConf(GlutenConfig.COLUMNAR_WHOLESTAGE_FALLBACK_THRESHOLD.key -> "1") { val cached = spark.table("lineitem").cache() @@ -143,4 +165,24 @@ class VeloxColumnarCacheSuite extends VeloxWholeStageTransformerSuite with Adapt } } } + + test("Fix cache output if selectedAttributes has wrong ordering with cacheAttributes") { + withTempPath { + path => + spark + .range(10) + .selectExpr("id as c1", "id % 3 as c2", "id % 5 as c3") + .write + .parquet(path.getCanonicalPath) + + val df = spark.read.parquet(path.getCanonicalPath) + val expected = df.select("c3", "c2", "c1").collect() + try { + val result = df.cache().select("c3", "c2", "c1") + checkAnswer(result, expected) + } finally { + df.unpersist() + } + } + } } diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxDataTypeValidationSuite.scala b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxDataTypeValidationSuite.scala index 02b385fc2ece..130a05f90194 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxDataTypeValidationSuite.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxDataTypeValidationSuite.scala @@ -262,7 +262,7 @@ class VeloxDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { // Validation: Window. runQueryAndCompare( - "select row_number() over (partition by date order by date) from type1 order by int, date") { + "select row_number() over (partition by date order by int) from type1 order by int, date") { checkOperatorMatch[WindowExecTransformer] } @@ -445,7 +445,7 @@ class VeloxDataTypeValidationSuite extends VeloxWholeStageTransformerSuite { } } - test("Velox Parquet Write") { + ignore("Velox Parquet Write") { withSQLConf(("spark.gluten.sql.native.writer.enabled", "true")) { withTempDir { dir => diff --git a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala index b4347e794a57..c306d70ac519 100644 --- a/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala +++ b/backends-velox/src/test/scala/io/glutenproject/execution/VeloxStringFunctionsSuite.scala @@ -583,4 +583,18 @@ class VeloxStringFunctionsSuite extends VeloxWholeStageTransformerSuite { s"select l_orderkey, substring(l_comment, $NULL_STR_COL, 3) " + s"from $LINEITEM_TABLE limit $LENGTH")(checkOperatorMatch[ProjectExecTransformer]) } + + test("left") { + runQueryAndCompare( + s"select l_orderkey, left(l_comment, 1) " + + s"from $LINEITEM_TABLE limit $LENGTH")(checkOperatorMatch[ProjectExecTransformer]) + + runQueryAndCompare( + s"select l_orderkey, left($NULL_STR_COL, 1) " + + s"from $LINEITEM_TABLE limit $LENGTH")(checkOperatorMatch[ProjectExecTransformer]) + + runQueryAndCompare( + s"select l_orderkey, left(l_comment, $NULL_STR_COL) " + + s"from $LINEITEM_TABLE limit $LENGTH")(checkOperatorMatch[ProjectExecTransformer]) + } } 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 e674d07d0a43..c11633038582 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 @@ -97,7 +97,7 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { _.getMessage.toString.contains("Use Gluten partition write for hive")) == native) } - test("test hive static partition write table") { + ignore("test hive static partition write table") { withTable("t") { spark.sql( "CREATE TABLE t (c int, d long, e long)" + @@ -127,7 +127,7 @@ class VeloxParquetWriteForHiveSuite extends GlutenQueryTest with SQLTestUtils { } } - test("test hive write table") { + ignore("test hive write table") { withTable("t") { spark.sql("CREATE TABLE t (c int) STORED AS PARQUET") withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { 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 a8549fafb6f2..535cf6354c1b 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 @@ -38,9 +38,9 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { super.sparkConf.set("spark.gluten.sql.native.writer.enabled", "true") } - test("test write parquet with compression codec") { + ignore("test write parquet with compression codec") { // compression codec details see `VeloxParquetDatasource.cc` - Seq("snappy", "gzip", "zstd", "none", "uncompressed") + Seq("snappy", "gzip", "zstd", "lz4", "none", "uncompressed") .foreach { codec => val extension = codec match { @@ -71,7 +71,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { } } - test("test ctas") { + ignore("test ctas") { withTable("velox_ctas") { spark .range(100) @@ -82,7 +82,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { } } - test("test parquet dynamic partition write") { + ignore("test parquet dynamic partition write") { withTempPath { f => val path = f.getCanonicalPath @@ -108,7 +108,7 @@ class VeloxParquetWriteSuite extends VeloxWholeStageTransformerSuite { } } - test("parquet write with empty dataframe") { + ignore("parquet write with empty dataframe") { withTempPath { f => val df = spark.emptyDataFrame.select(lit(1).as("i")) diff --git a/cpp-ch/clickhouse.version b/cpp-ch/clickhouse.version index 101b818fabc0..017a36f3e359 100644 --- a/cpp-ch/clickhouse.version +++ b/cpp-ch/clickhouse.version @@ -1,3 +1,3 @@ CH_ORG=Kyligence -CH_BRANCH=rebase_ch/20231007 -CH_COMMIT=253a035074b +CH_BRANCH=rebase_ch/20231117 +CH_COMMIT=61782b56dc4 diff --git a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.h b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.h index 94883933de76..767ba0550c0b 100644 --- a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.h +++ b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionGroupBloomFilter.h @@ -126,6 +126,11 @@ class AggregateFunctionGroupBloomFilter final : public IAggregateFunctionDataHel void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { + // Skip un-initted values + if (!this->data(rhs).initted) + { + return; + } const auto & bloom_other = this->data(rhs).bloom_filter; const auto & filter_other = bloom_other.getFilter(); if (!this->data(place).initted) diff --git a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionPartialMerge.cpp b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionPartialMerge.cpp index a481532d4bce..0ecb2941004c 100644 --- a/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionPartialMerge.cpp +++ b/cpp-ch/local-engine/AggregateFunctions/AggregateFunctionPartialMerge.cpp @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include +#include #include #include diff --git a/cpp-ch/local-engine/Builder/SerializedPlanBuilder.cpp b/cpp-ch/local-engine/Builder/SerializedPlanBuilder.cpp index 22499794a500..92e5c564110d 100644 --- a/cpp-ch/local-engine/Builder/SerializedPlanBuilder.cpp +++ b/cpp-ch/local-engine/Builder/SerializedPlanBuilder.cpp @@ -259,7 +259,7 @@ std::shared_ptr SerializedPlanBuilder::buildType(const DB::Data res->mutable_i32()->set_nullability(type_nullability); else if (which.isInt64()) res->mutable_i64()->set_nullability(type_nullability); - else if (which.isString() || which.isAggregateFunction()) + else if (which.isStringOrFixedString() || which.isAggregateFunction()) res->mutable_binary()->set_nullability(type_nullability); /// Spark Binary type is more similiar to CH String type else if (which.isFloat32()) res->mutable_fp32()->set_nullability(type_nullability); diff --git a/cpp-ch/local-engine/CMakeLists.txt b/cpp-ch/local-engine/CMakeLists.txt index a93a9eaf710f..251950f09669 100644 --- a/cpp-ch/local-engine/CMakeLists.txt +++ b/cpp-ch/local-engine/CMakeLists.txt @@ -13,6 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w -ffunction-sections -fdata-sections") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -w -ffunction-sections -fdata-sections") +set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-export-dynamic -Wl,--gc-sections") + if (COMPILER_CLANG AND CMAKE_CXX_COMPILER_VERSION VERSION_LESS 16) set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_WRAPPER}") else() @@ -56,6 +60,7 @@ include_directories( ${ClickHouse_SOURCE_DIR}/src ${ClickHouse_SOURCE_DIR}/base ${ClickHouse_SOURCE_DIR}/contrib/orc/c++/include + ${ClickHouse_SOURCE_DIR}/contrib/arrow-cmake/cpp/src/orc/c++/include ${CMAKE_BINARY_DIR}/contrib/orc/c++/include ${ClickHouse_SOURCE_DIR}/contrib/azure/sdk/storage/azure-storage-blobs/inc ${ClickHouse_SOURCE_DIR}/contrib/azure/sdk/core/azure-core/inc diff --git a/cpp-ch/local-engine/Common/CHUtil.cpp b/cpp-ch/local-engine/Common/CHUtil.cpp index dba8205da03e..9c3d1c64e008 100644 --- a/cpp-ch/local-engine/Common/CHUtil.cpp +++ b/cpp-ch/local-engine/Common/CHUtil.cpp @@ -17,7 +17,7 @@ #include #include #include -#include +#include #include #include #include @@ -398,9 +398,9 @@ const DB::ActionsDAG::Node * ActionsDAGUtil::convertNodeType( type_name_col.type = std::make_shared(); const auto * right_arg = &actions_dag->addColumn(std::move(type_name_col)); const auto * left_arg = node; - DB::FunctionCastBase::Diagnostic diagnostic = {node->result_name, node->result_name}; + DB::CastDiagnostic diagnostic = {node->result_name, node->result_name}; DB::FunctionOverloadResolverPtr func_builder_cast - = DB::CastInternalOverloadResolver::createImpl(std::move(diagnostic)); + = DB::createInternalCastOverloadResolver(DB::CastType::nonAccurate, std::move(diagnostic)); DB::ActionsDAG::NodeRawConstPtrs children = {left_arg, right_arg}; return &actions_dag->addFunction(func_builder_cast, std::move(children), result_name); @@ -438,7 +438,9 @@ std::map BackendInitializerUtil::getBackendConfMap(std namespace pb_util = google::protobuf::util; pb_util::JsonOptions options; std::string json; - pb_util::MessageToJsonString(*plan_ptr, &json, options); + auto s = pb_util::MessageToJsonString(*plan_ptr, &json, options); + if (!s.ok()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert Substrait Plan to Json"); LOG_DEBUG(&Poco::Logger::get("CHUtil"), "Update Config Map Plan:\n{}", json); } @@ -598,6 +600,8 @@ void BackendInitializerUtil::initSettings(std::map & b settings.set("input_format_parquet_import_nested", true); settings.set("input_format_json_read_numbers_as_strings", true); settings.set("input_format_json_read_bools_as_numbers", false); + settings.set("input_format_csv_trim_whitespaces", false); + settings.set("input_format_csv_allow_cr_end_of_line", true); settings.set("output_format_orc_string_as_string", true); settings.set("output_format_parquet_version", "1.0"); settings.set("output_format_parquet_compression_method", "snappy"); @@ -605,10 +609,10 @@ void BackendInitializerUtil::initSettings(std::map & b settings.set("output_format_parquet_fixed_string_as_fixed_byte_array", false); settings.set("output_format_json_quote_64bit_integers", false); settings.set("output_format_json_quote_denormals", true); + settings.set("output_format_json_skip_null_value_in_named_tuples", true); settings.set("function_json_value_return_type_allow_complex", true); settings.set("function_json_value_return_type_allow_nullable", true); - - + settings.set("precise_float_parsing", true); } void BackendInitializerUtil::initContexts(DB::Context::ConfigurationPtr config) diff --git a/cpp-ch/local-engine/Common/CHUtil.h b/cpp-ch/local-engine/Common/CHUtil.h index 7059ea73f29f..93061c968587 100644 --- a/cpp-ch/local-engine/Common/CHUtil.h +++ b/cpp-ch/local-engine/Common/CHUtil.h @@ -125,6 +125,9 @@ class BackendInitializerUtil // use excel text parser inline static const std::string USE_EXCEL_PARSER = "use_excel_serialization"; + inline static const std::string EXCEL_EMPTY_AS_NULL = "use_excel_serialization.empty_as_null"; + inline static const std::string EXCEL_NUMBER_FORCE = "use_excel_serialization.number_force"; + inline static const std::string EXCEL_QUOTE_STRICT = "use_excel_serialization.quote_strict"; inline static const String CH_BACKEND_PREFIX = "spark.gluten.sql.columnar.backend.ch"; inline static const String CH_RUNTIME_CONFIG = "runtime_config"; diff --git a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp index 67c27961a7f4..a19fe3ec8611 100644 --- a/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp +++ b/cpp-ch/local-engine/Common/GlutenSignalHandler.cpp @@ -32,7 +32,7 @@ #include #include -#include "config_version.h" +#include using namespace local_engine; diff --git a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h index 6bd3b04eaeee..01d628c5c63a 100644 --- a/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h +++ b/cpp-ch/local-engine/Functions/SparkFunctionRoundHalfUp.h @@ -19,50 +19,28 @@ #include -namespace local_engine +namespace local_engine { using namespace DB; - -/// Implementation for round half up. Not vectorized. - -inline float roundHalfUp(float x) -{ - return roundf(x); - - UNREACHABLE(); -} - -inline double roundHalfUp(double x) -{ - return round(x); - - UNREACHABLE(); -} - template class BaseFloatRoundingHalfUpComputation { public: using ScalarType = T; - using VectorType = T; + using VectorType = Float64; 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 load(const ScalarType * in) { return static_cast(*in); } + static VectorType load1(ScalarType in) { return in; } + static ScalarType store(ScalarType * out, VectorType val) { return *out = static_cast(val); } static VectorType multiply(VectorType val, VectorType scale) { return val * scale; } static VectorType divide(VectorType val, VectorType scale) { return val / scale; } - static VectorType apply(VectorType val){return roundHalfUp(val);} - - static VectorType prepare(size_t scale) - { - return load1(scale); - } + static VectorType apply(VectorType val) { return round(val); } + static VectorType prepare(size_t scale) { return load1(scale); } }; - /** Implementation of low-level round-off functions for floating-point values. */ template @@ -140,9 +118,6 @@ struct FloatRoundingHalfUpImpl }; - - - /** Select the appropriate processing algorithm depending on the scale. */ template diff --git a/cpp-ch/local-engine/Functions/SparkFunctionToDate.cpp b/cpp-ch/local-engine/Functions/SparkFunctionToDate.cpp new file mode 100644 index 000000000000..d2744de2051e --- /dev/null +++ b/cpp-ch/local-engine/Functions/SparkFunctionToDate.cpp @@ -0,0 +1,161 @@ +/* + * 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 + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} +} + +namespace local_engine +{ +class SparkFunctionConvertToDate : public DB::FunctionToDate32OrNull +{ +public: + static constexpr auto name = "spark_to_date"; + static DB::FunctionPtr create(DB::ContextPtr) { return std::make_shared(); } + SparkFunctionConvertToDate() = default; + ~SparkFunctionConvertToDate() override = default; + DB::String getName() const override { return name; } + + bool checkDateFormat(DB::ReadBuffer & buf) const + { + auto checkNumbericASCII = [&](DB::ReadBuffer & rb, size_t start, size_t length) -> bool + { + for (size_t i = start; i < start + length; ++i) + { + if (!isNumericASCII(*(rb.position() + i))) + return false; + } + return true; + }; + auto checkDelimiter = [&](DB::ReadBuffer & rb, size_t pos) -> bool + { + if (*(rb.position() + pos) != '-') + return false; + else + return true; + }; + if (!checkNumbericASCII(buf, 0, 4) + || !checkDelimiter(buf, 4) + || !checkNumbericASCII(buf, 5, 2) + || !checkDelimiter(buf, 7) + || !checkNumbericASCII(buf, 8, 2)) + return false; + else + { + int month = (*(buf.position() + 5) - '0') * 10 + (*(buf.position() + 6) - '0'); + if (month <= 0 || month > 12) + return false; + int day = (*(buf.position() + 8) - '0') * 10 + (*(buf.position() + 9) - '0'); + if (day <= 0 || day > 31) + return false; + else if (day == 31 && (month == 2 || month == 4 || month == 6 || month == 9 || month == 11)) + return false; + else if (day == 30 && month == 2) + return false; + else + { + int year = (*(buf.position() + 0) - '0') * 1000 + + (*(buf.position() + 1) - '0') * 100 + + (*(buf.position() + 2) - '0') * 10 + + (*(buf.position() + 3) - '0'); + if (day == 29 && month == 2 && year % 4 != 0) + return false; + else + return true; + } + } + } + + DB::ColumnPtr executeImpl(const DB::ColumnsWithTypeAndName & arguments, const DB::DataTypePtr & result_type, size_t) const override + { + if (arguments.size() != 1) + throw DB::Exception(DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 1.", name); + + const DB::ColumnWithTypeAndName arg1 = arguments[0]; + const auto * src_col = checkAndGetColumn(arg1.column.get()); + size_t size = src_col->size(); + + if (!result_type->isNullable()) + throw DB::Exception(DB::ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s return type must be nullable", name); + + if (!isDate32(removeNullable(result_type))) + throw DB::Exception(DB::ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s return type must be data32.", name); + + using ColVecTo = DB::DataTypeDate32::ColumnType; + typename ColVecTo::MutablePtr result_column = ColVecTo::create(size); + typename ColVecTo::Container & result_container = result_column->getData(); + DB::ColumnUInt8::MutablePtr null_map = DB::ColumnUInt8::create(size); + typename DB::ColumnUInt8::Container & null_container = null_map->getData(); + const DateLUTImpl * utc_time_zone = &DateLUT::instance("UTC"); + + for (size_t i = 0; i < size; ++i) + { + auto str = src_col->getDataAt(i); + if (str.size < 10) + { + null_container[i] = true; + result_container[i] = 0; + continue; + } + else + { + DB::ReadBufferFromMemory buf(str.data, str.size); + while(!buf.eof() && *buf.position() == ' ') + { + buf.position() ++; + } + if(buf.buffer().end() - buf.position() < 10) + { + null_container[i] = true; + result_container[i] = 0; + continue; + } + if (!checkDateFormat(buf)) + { + null_container[i] = true; + result_container[i] = 0; + } + else + { + bool parsed = tryParseImpl(result_container[i], buf, utc_time_zone, false); + null_container[i] = !parsed; + } + } + } + return DB::ColumnNullable::create(std::move(result_column), std::move(null_map)); + } +}; + +REGISTER_FUNCTION(SparkToDate) +{ + factory.registerFunction(); +} + +} diff --git a/cpp-ch/local-engine/Functions/SparkParseURL.cpp b/cpp-ch/local-engine/Functions/SparkParseURL.cpp index 5023e7c37d66..c39c8eaa90f8 100644 --- a/cpp-ch/local-engine/Functions/SparkParseURL.cpp +++ b/cpp-ch/local-engine/Functions/SparkParseURL.cpp @@ -14,23 +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 -#include #include -#include #include namespace DB diff --git a/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp b/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp index 3ef616b6ce0f..83c37c7ad752 100644 --- a/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp +++ b/cpp-ch/local-engine/Join/StorageJoinFromReadBuffer.cpp @@ -16,7 +16,7 @@ */ #include "StorageJoinFromReadBuffer.h" -#include +#include #include #include #include @@ -42,7 +42,7 @@ using namespace DB; void restore(DB::ReadBuffer & in, IJoin & join, const Block & sample_block) { - NativeReader block_stream(in, 0); + local_engine::NativeReader block_stream(in); ProfileInfo info; { diff --git a/cpp-ch/local-engine/Operator/ExpandTransform.cpp b/cpp-ch/local-engine/Operator/ExpandTransform.cpp index 0e8e5c7236c1..7a871111e9d7 100644 --- a/cpp-ch/local-engine/Operator/ExpandTransform.cpp +++ b/cpp-ch/local-engine/Operator/ExpandTransform.cpp @@ -27,6 +27,14 @@ #include #include +namespace DB +{ +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} +} + namespace local_engine { ExpandTransform::ExpandTransform(const DB::Block & input_, const DB::Block & output_, const ExpandField & project_set_exprs_) @@ -53,7 +61,8 @@ ExpandTransform::Status ExpandTransform::prepare() if (has_output) { - output.push(nextChunk()); + output.push(std::move(output_chunk)); + has_output = false; return Status::PortFull; } @@ -72,6 +81,7 @@ ExpandTransform::Status ExpandTransform::prepare() input_chunk = input.pull(true); has_input = true; + expand_expr_iterator = 0; } return Status::Ready; @@ -79,73 +89,57 @@ ExpandTransform::Status ExpandTransform::prepare() void ExpandTransform::work() { - assert(expanded_chunks.empty()); + if (expand_expr_iterator >= project_set_exprs.getExpandRows()) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "expand_expr_iterator >= project_set_exprs.getExpandRows()"); const auto & original_cols = input_chunk.getColumns(); size_t rows = input_chunk.getNumRows(); - - for (size_t i = 0; i < project_set_exprs.getExpandRows(); ++i) + DB::Columns cols; + for (size_t j = 0; j < project_set_exprs.getExpandCols(); ++j) { - DB::Columns cols; - for (size_t j = 0; j < project_set_exprs.getExpandCols(); ++j) - { - const auto & type = project_set_exprs.getTypes()[j]; - const auto & kind = project_set_exprs.getKinds()[i][j]; - const auto & field = project_set_exprs.getFields()[i][j]; + const auto & type = project_set_exprs.getTypes()[j]; + const auto & kind = project_set_exprs.getKinds()[expand_expr_iterator][j]; + const auto & field = project_set_exprs.getFields()[expand_expr_iterator][j]; - if (kind == EXPAND_FIELD_KIND_SELECTION) + if (kind == EXPAND_FIELD_KIND_SELECTION) + { + const auto & original_col = original_cols[field.get()]; + if (type->isNullable() == original_col->isNullable()) + { + cols.push_back(original_col); + } + else if (type->isNullable() && !original_col->isNullable()) { - const auto & original_col = original_cols[field.get()]; - if (type->isNullable() == original_col->isNullable()) - { - cols.push_back(original_col); - } - else if (type->isNullable() && !original_col->isNullable()) - { - auto null_map = DB::ColumnUInt8::create(rows, 0); - auto col = DB::ColumnNullable::create(original_col, std::move(null_map)); - cols.push_back(std::move(col)); - } - else - { - throw DB::Exception( - DB::ErrorCodes::LOGICAL_ERROR, - "Miss match nullable, column {} is nullable, but type {} is not nullable", - original_col->getName(), - type->getName()); - } + auto null_map = DB::ColumnUInt8::create(rows, 0); + auto col = DB::ColumnNullable::create(original_col, std::move(null_map)); + cols.push_back(std::move(col)); } else { - if (field.isNull()) - { - // Add null column - auto null_map = DB::ColumnUInt8::create(rows, 1); - auto nested_type = DB::removeNullable(type); - auto col = DB::ColumnNullable::create(nested_type->createColumn()->cloneResized(rows), std::move(null_map)); - cols.push_back(std::move(col)); - } - else - { - // Add constant column: gid, gpos, etc. - auto col = type->createColumnConst(rows, field); - cols.push_back(col->convertToFullColumnIfConst()); - } + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Miss match nullable, column {} is nullable, but type {} is not nullable", + original_col->getName(), + type->getName()); } } - expanded_chunks.push_back(DB::Chunk(cols, rows)); + else if (field.isNull()) + { + // Add null column + auto null_map = DB::ColumnUInt8::create(rows, 1); + auto nested_type = DB::removeNullable(type); + auto col = DB::ColumnNullable::create(nested_type->createColumn()->cloneResized(rows), std::move(null_map)); + cols.push_back(std::move(col)); + } + else + { + // Add constant column: gid, gpos, etc. + auto col = type->createColumnConst(rows, field); + cols.push_back(col->convertToFullColumnIfConst()); + } } - - has_output = true; - has_input = false; -} - -DB::Chunk ExpandTransform::nextChunk() -{ - assert(!expanded_chunks.empty()); - DB::Chunk ret; - ret.swap(expanded_chunks.front()); - expanded_chunks.pop_front(); - has_output = !expanded_chunks.empty(); - return ret; + output_chunk = DB::Chunk(cols, rows); + expand_expr_iterator += 1; + has_output = expand_expr_iterator <= project_set_exprs.getExpandRows(); + has_input = expand_expr_iterator < project_set_exprs.getExpandRows(); } } diff --git a/cpp-ch/local-engine/Operator/ExpandTransorm.h b/cpp-ch/local-engine/Operator/ExpandTransorm.h index c80a1b39ccca..16669c2873f4 100644 --- a/cpp-ch/local-engine/Operator/ExpandTransorm.h +++ b/cpp-ch/local-engine/Operator/ExpandTransorm.h @@ -46,9 +46,9 @@ class ExpandTransform : public DB::IProcessor ExpandField project_set_exprs; bool has_input = false; bool has_output = false; + size_t expand_expr_iterator = 0; DB::Chunk input_chunk; - std::list expanded_chunks; - DB::Chunk nextChunk(); + DB::Chunk output_chunk; }; } diff --git a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp index 98b8286a09f0..961a5e7ab8ef 100644 --- a/cpp-ch/local-engine/Parser/AggregateRelParser.cpp +++ b/cpp-ch/local-engine/Parser/AggregateRelParser.cpp @@ -16,7 +16,7 @@ */ #include "AggregateRelParser.h" #include -#include +#include #include #include #include diff --git a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp index ae97fdd70850..89903b23ad06 100644 --- a/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp +++ b/cpp-ch/local-engine/Parser/SerializedPlanParser.cpp @@ -61,9 +61,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -332,7 +334,7 @@ DB::QueryPlanPtr SerializedPlanParser::parseMergeTreeTable(const substrait::Read "", MergeTreeData::MergingParams(), buildMergeTreeSettings()); - custom_storage_merge_tree->loadDataParts(false); + custom_storage_merge_tree->loadDataParts(false, std::nullopt); return custom_storage_merge_tree; }); query_context.storage_snapshot = std::make_shared(*storage, metadata); @@ -367,7 +369,7 @@ DB::QueryPlanPtr SerializedPlanParser::parseMergeTreeTable(const substrait::Read query_context.storage_snapshot, *query_info, context, - 4096 * 2, + context->getSettingsRef().max_block_size, 1); QueryPlanPtr query = std::make_unique(); steps.emplace_back(read_step.get()); @@ -436,7 +438,9 @@ QueryPlanPtr SerializedPlanParser::parse(std::unique_ptr plan) namespace pb_util = google::protobuf::util; pb_util::JsonOptions options; std::string json; - pb_util::MessageToJsonString(*plan, &json, options); + auto s = pb_util::MessageToJsonString(*plan, &json, options); + if (!s.ok()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Can not convert Substrait Plan to Json"); LOG_DEBUG(&Poco::Logger::get("SerializedPlanParser"), "substrait plan:\n{}", json); } parseExtensions(plan->extensions()); @@ -1654,45 +1658,11 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act /// FIXME. Now we treet '1900-01-01' as null value. Not realy good. /// Updating `toDate32OrNull` to return null if the string is invalid is not acceptable by /// ClickHouse (https://github.com/ClickHouse/ClickHouse/issues/47120). - - /// isNotNull(toDate32OrNull(date)) - String to_date_function_name = "toDate32OrNull"; - const auto * date_node = toFunctionNode(actions_dag, to_date_function_name, args); - const auto * date_is_not_null_node = toFunctionNode(actions_dag, "isNotNull", {date_node}); - - /// isNotNull(toDate32(parseDateTimeOrNull(substring(trimLeft(date), 1, 10), '%Y-%m-%d')) - const auto * substr_offset_node = add_column(std::make_shared(), 1); - const auto * substr_length_node = add_column(std::make_shared(), 10); - const auto * trim_string_node = toFunctionNode(actions_dag, "trimLeft", {args[0]}); - const auto * substr_node = toFunctionNode(actions_dag, "substring", {trim_string_node, substr_offset_node, substr_length_node}); - const auto * date_format_node = add_column(std::make_shared(), "%Y-%m-%d"); - const auto * parse_date_node = toFunctionNode(actions_dag, "parseDateTimeOrNull", {substr_node, date_format_node}); - const auto * parse_date_is_not_null_node = toFunctionNode(actions_dag, "isNotNull", {parse_date_node}); - - /// toDate32(parseDateTimeOrNull(substring(trimLeft(date), 1, 10), '%Y-%m-%d')) - const auto * date_node_from_parse = toFunctionNode(actions_dag, "toDate32", {parse_date_node}); - /// const null node - const auto * null_const_node = add_column(makeNullable(std::make_shared()), Field{}); - - /** - * Parse toDate(s) as - * if (isNotNull(toDate32OrNull)) - * toDate32OrNull(s) - * else if (isNotNull(parseDateTimeOrNull(substring(trimLeft(s)), 1, 10), '%Y-%m-%d')) - * toDate32(parseDateTimeOrNull(substring(trimLeft(s)), 1, 10), '%Y-%m-%d')) - * else - * null - */ - const auto * to_date_multi_if_node = toFunctionNode(actions_dag, "multiIf", { - date_is_not_null_node, - date_node, - parse_date_is_not_null_node, - date_node_from_parse, - null_const_node - }); + String function_name = "spark_to_date"; + const auto * date_node = toFunctionNode(actions_dag, function_name, args); const auto * zero_date_col_node = add_column(std::make_shared(), "1900-01-01"); - const auto * zero_date_node = toFunctionNode(actions_dag, to_date_function_name, {zero_date_col_node}); - DB::ActionsDAG::NodeRawConstPtrs nullif_args = {to_date_multi_if_node, zero_date_node}; + const auto * zero_date_node = toFunctionNode(actions_dag, function_name, {zero_date_col_node}); + DB::ActionsDAG::NodeRawConstPtrs nullif_args = {date_node, zero_date_node}; function_node = toFunctionNode(actions_dag, "nullIf", nullif_args); } else if (substrait_type.has_binary()) @@ -1708,8 +1678,10 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act const auto * final_arg_node = if_not_finite_node; if (args[0]->result_type->isNullable()) { + DB::Field null_field; + const auto * null_value = add_column(args[0]->result_type, null_field); const auto * is_null_node = toFunctionNode(actions_dag, "isNull", {args[0]}); - const auto * if_node = toFunctionNode(actions_dag, "if", {is_null_node, args[0], if_not_finite_node}); + const auto * if_node = toFunctionNode(actions_dag, "if", {is_null_node, null_value, if_not_finite_node}); final_arg_node = if_node; } function_node = toFunctionNode( @@ -1837,6 +1809,7 @@ const ActionsDAG::Node * SerializedPlanParser::parseExpression(ActionsDAGPtr act DB::ActionsDAG::NodeRawConstPtrs cast_args({function_node, add_column(type, true), add_column(type, Field())}); auto cast = FunctionFactory::instance().get("if", context); function_node = toFunctionNode(actions_dag, "if", cast_args); + actions_dag->addOrReplaceInOutputs(*function_node); } return function_node; } @@ -1878,7 +1851,9 @@ QueryPlanPtr SerializedPlanParser::parse(const std::string & plan) QueryPlanPtr SerializedPlanParser::parseJson(const std::string & json_plan) { auto plan_ptr = std::make_unique(); - google::protobuf::util::JsonStringToMessage(google::protobuf::stringpiece_internal::StringPiece(json_plan.c_str()), plan_ptr.get()); + auto s = google::protobuf::util::JsonStringToMessage(absl::string_view(json_plan.c_str()), plan_ptr.get()); + if (!s.ok()) + throw Exception(ErrorCodes::CANNOT_PARSE_PROTOBUF_SCHEMA, "Parse substrait::Plan from json string failed: {}", s.ToString()); return parse(std::move(plan_ptr)); } @@ -2263,21 +2238,19 @@ bool LocalExecutor::checkAndSetDefaultBlock(size_t current_block_columns, bool h { return has_next_blocks; } - auto cols = currentBlock().getColumnsWithTypeAndName(); - for (const auto & col : cols) + bool should_set_default_value = false; + for (auto p : query_pipeline.getProcessors()) { - String col_name = col.name; - DataTypePtr col_type = col.type; - if (col_name.compare(0, 4, "sum#") != 0 && col_name.compare(0, 4, "max#") != 0 && col_name.compare(0, 4, "min#") != 0 - && col_name.compare(0, 6, "count#") != 0) + if (p->getName() == "MergingAggregatedTransform") { - return false; - } - if (!isInteger(col_type) && !col_type->isNullable()) - { - return false; + DB::MergingAggregatedStep * agg_step = static_cast(p->getQueryPlanStep()); + auto query_params = agg_step->getParams(); + should_set_default_value = query_params.keys_size == 0; } } + if (!should_set_default_value) + return false; + auto cols = currentBlock().getColumnsWithTypeAndName(); for (size_t i = 0; i < cols.size(); i++) { const DB::ColumnWithTypeAndName col = cols[i]; diff --git a/cpp-ch/local-engine/Parser/WindowRelParser.cpp b/cpp-ch/local-engine/Parser/WindowRelParser.cpp index 6d1f49d3e280..2fb9debbb531 100644 --- a/cpp-ch/local-engine/Parser/WindowRelParser.cpp +++ b/cpp-ch/local-engine/Parser/WindowRelParser.cpp @@ -84,7 +84,7 @@ WindowRelParser::parse(DB::QueryPlanPtr current_plan_, const substrait::Rel & re { auto & win = it.second; - auto window_step = std::make_unique(current_plan->getCurrentDataStream(), win, win.window_functions); + auto window_step = std::make_unique(current_plan->getCurrentDataStream(), win, win.window_functions, false); window_step->setStepDescription("Window step for window '" + win.window_name + "'"); steps.emplace_back(window_step.get()); current_plan->addStep(std::move(window_step)); diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.cpp b/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.cpp index ce0cb5251c2c..fc334a2146db 100644 --- a/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.cpp +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.cpp @@ -14,27 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include #include namespace local_engine { -#define REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(cls_name, substait_name, ch_name) \ - class AggregateFunctionParser##cls_name : public AggregateFunctionParser \ - { \ - public: \ - AggregateFunctionParser##cls_name(SerializedPlanParser * plan_parser_) : AggregateFunctionParser(plan_parser_) \ - { \ - } \ - ~AggregateFunctionParser##cls_name() override = default; \ - String getName() const override { return #substait_name; } \ - static constexpr auto name = #substait_name; \ - String getCHFunctionName(const CommonFunctionInfo &) const override { return #ch_name; } \ - String getCHFunctionName(const DB::DataTypes &) const override { return #ch_name; } \ - }; \ - static const AggregateFunctionParserRegister register_##cls_name = AggregateFunctionParserRegister(); - REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(Sum, sum, sum) REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(Avg, avg, avg) REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(Min, min, min) diff --git a/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.h b/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.h new file mode 100644 index 000000000000..c486e16e1c69 --- /dev/null +++ b/cpp-ch/local-engine/Parser/aggregate_function_parser/CommonAggregateFunctionParser.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 + + +namespace local_engine +{ + +#define REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(cls_name, substait_name, ch_name) \ + class AggregateFunctionParser##cls_name : public AggregateFunctionParser \ + { \ + public: \ + AggregateFunctionParser##cls_name(SerializedPlanParser * plan_parser_) : AggregateFunctionParser(plan_parser_) \ + { \ + } \ + ~AggregateFunctionParser##cls_name() override = default; \ + String getName() const override { return #substait_name; } \ + static constexpr auto name = #substait_name; \ + String getCHFunctionName(const CommonFunctionInfo &) const override { return #ch_name; } \ + String getCHFunctionName(const DB::DataTypes &) const override { return #ch_name; } \ + }; \ + static const AggregateFunctionParserRegister register_##cls_name = AggregateFunctionParserRegister(); + +} diff --git a/cpp-ch/local-engine/Parser/example_udf/customSum.cpp b/cpp-ch/local-engine/Parser/example_udf/customSum.cpp new file mode 100644 index 000000000000..66328495d0e2 --- /dev/null +++ b/cpp-ch/local-engine/Parser/example_udf/customSum.cpp @@ -0,0 +1,28 @@ +/* + * 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 local_engine +{ +// Only for ut to test custom aggregate function +REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(CustomSum, custom_sum, sum) +REGISTER_COMMON_AGGREGATE_FUNCTION_PARSER(CustomSumDouble, custom_sum_double, sum) +} 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 b58217a93a68..956ef162f1a9 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/elementAt.cpp @@ -16,6 +16,8 @@ */ #include +#include +#include namespace local_engine { @@ -26,6 +28,19 @@ namespace local_engine ~FunctionParserElementAt() override = default; static constexpr auto name = "element_at"; String getName() const override { return name; } + + const ActionsDAG::Node * parse( + const substrait::Expression_ScalarFunction & substrait_func, + ActionsDAGPtr & actions_dag) const override + { + auto parsed_args = parseFunctionArguments(substrait_func, "", actions_dag); + if (parsed_args.size() != 2) + throw Exception(DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires exactly two arguments", getName()); + if (isMap(removeNullable(parsed_args[0]->result_type))) + return toFunctionNode(actions_dag, "arrayElement", parsed_args); + else + return FunctionParserArrayElement::parse(substrait_func, actions_dag); + } }; static FunctionParserRegister register_element_at; diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h index 59e47011091a..9d8aae8e21a6 100644 --- a/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/parseUrl.h @@ -22,7 +22,7 @@ class ParseURLParser final : public FunctionParser { public: static constexpr auto name = "parse_url"; - ParseURLParser(SerializedPlanParser * plan_parser) : FunctionParser(plan_parser) { } + ParseURLParser(SerializedPlanParser * plan_parser_) : FunctionParser(plan_parser_) { } ~ParseURLParser() override = default; String getName() const override { return name; } diff --git a/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp b/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp new file mode 100644 index 000000000000..ebe0e6adff69 --- /dev/null +++ b/cpp-ch/local-engine/Parser/scalar_function_parser/substring.cpp @@ -0,0 +1,78 @@ +/* + * 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 + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} +} + +namespace local_engine +{ +class FunctionParserSubstring : public FunctionParser +{ +public: + explicit FunctionParserSubstring(SerializedPlanParser * plan_parser_) : FunctionParser(plan_parser_) { } + ~FunctionParserSubstring() override = default; + static constexpr auto name = "substring"; + String getName() const override { return name; } + + const ActionsDAG::Node * parse( + const substrait::Expression_ScalarFunction & substrait_func, + ActionsDAGPtr & actions_dag) const override + { + auto parsed_args = parseFunctionArguments(substrait_func, "", actions_dag); + if (parsed_args.size() != 2 && parsed_args.size() != 3) + throw Exception(DB::ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires two or three arguments", getName()); + DB::DataTypePtr start_index_data_type = removeNullable(parsed_args[1]->result_type); + if (!isInteger(start_index_data_type)) + throw Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Function {}'s second arguments must be int type"); + /** + parse substring(str, start_index, length) as + if (start_index == 0) + substring(str, start_index+1, length) + else + substring(str, start_index, length) + */ + auto * const_zero_node = addColumnToActionsDAG(actions_dag, start_index_data_type, Field(0)); + auto * const_one_node = addColumnToActionsDAG(actions_dag, start_index_data_type, Field(1)); + auto * equals_zero_node = toFunctionNode(actions_dag, "equals", {parsed_args[1], const_zero_node}); + auto * index_plus_node = toFunctionNode(actions_dag, "plus", {parsed_args[1], const_one_node}); + auto * if_node = toFunctionNode(actions_dag, "if", {equals_zero_node, index_plus_node, parsed_args[1]}); + const DB::ActionsDAG::Node * substring_func_node; + if (parsed_args.size() == 2) + substring_func_node = toFunctionNode(actions_dag, "substringUTF8", {parsed_args[0], if_node}); + else + substring_func_node = toFunctionNode(actions_dag, "substringUTF8", {parsed_args[0], if_node, parsed_args[2]}); + return convertNodeTypeIfNeeded(substrait_func, substring_func_node, actions_dag); + } +protected: + String getCHFunctionName(const substrait::Expression_ScalarFunction & /*substrait_func*/) const override + { + return "substringUTF8"; + } +}; + +static FunctionParserRegister register_substring; +} diff --git a/cpp-ch/local-engine/Rewriter/ExpressionRewriter.h b/cpp-ch/local-engine/Rewriter/ExpressionRewriter.h index 52b8e195f68a..8c0bc0e0d981 100644 --- a/cpp-ch/local-engine/Rewriter/ExpressionRewriter.h +++ b/cpp-ch/local-engine/Rewriter/ExpressionRewriter.h @@ -33,7 +33,7 @@ enum SelfDefinedFunctionReference class GetJsonObjectFunctionWriter : public RelRewriter { public: - GetJsonObjectFunctionWriter(SerializedPlanParser * parser) : RelRewriter(parser) {} + GetJsonObjectFunctionWriter(SerializedPlanParser * parser_) : RelRewriter(parser_) {} ~GetJsonObjectFunctionWriter() override = default; void rewrite(substrait::Rel & rel) override diff --git a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp index 4f3ae465078e..16d56e9bb8e8 100644 --- a/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/CachedShuffleWriter.cpp @@ -17,6 +17,7 @@ #include "CachedShuffleWriter.h" #include #include +#include #include #include #include @@ -50,7 +51,7 @@ CachedShuffleWriter::CachedShuffleWriter(const String & short_name, SplitOptions { hash_fields.push_back(std::stoi(expr)); } - partitioner = std::make_unique(options.partition_nums, hash_fields, "cityHash64"); + partitioner = std::make_unique(options.partition_nums, hash_fields, options_.hash_algorithm); } else if (short_name == "single") { @@ -94,6 +95,10 @@ CachedShuffleWriter::CachedShuffleWriter(const String & short_name, SplitOptions void CachedShuffleWriter::split(DB::Block & block) { initOutputIfNeeded(block); + Stopwatch split_time_watch; + split_time_watch.start(); + block = convertAggregateStateInBlock(block); + split_result.total_split_time += split_time_watch.elapsedNanoseconds(); Stopwatch compute_pid_time_watch; compute_pid_time_watch.start(); @@ -105,7 +110,6 @@ void CachedShuffleWriter::split(DB::Block & block) { out_block.insert(block.getByPosition(output_columns_indicies[col])); } - partition_writer->write(partition_info, out_block); if (options.spill_threshold > 0 && partition_writer->totalCacheSize() > options.spill_threshold) diff --git a/cpp-ch/local-engine/Shuffle/NativeSplitter.cpp b/cpp-ch/local-engine/Shuffle/NativeSplitter.cpp index 4b8b56f99c57..d05f6633ca8a 100644 --- a/cpp-ch/local-engine/Shuffle/NativeSplitter.cpp +++ b/cpp-ch/local-engine/Shuffle/NativeSplitter.cpp @@ -30,6 +30,7 @@ #include #include #include +#include namespace local_engine { @@ -209,7 +210,7 @@ HashNativeSplitter::HashNativeSplitter(NativeSplitter::Options options_, jobject output_columns_indicies.push_back(std::stoi(*iter)); } - selector_builder = std::make_unique(options.partition_nums, hash_fields, "cityHash64"); + selector_builder = std::make_unique(options.partition_nums, hash_fields, options_.hash_algorithm); } void HashNativeSplitter::computePartitionId(Block & block) diff --git a/cpp-ch/local-engine/Shuffle/NativeSplitter.h b/cpp-ch/local-engine/Shuffle/NativeSplitter.h index c883da4bfb4a..c30f235b6550 100644 --- a/cpp-ch/local-engine/Shuffle/NativeSplitter.h +++ b/cpp-ch/local-engine/Shuffle/NativeSplitter.h @@ -41,6 +41,7 @@ class NativeSplitter : BlockIterator size_t partition_nums; std::string exprs_buffer; std::string schema_buffer; + std::string hash_algorithm; }; struct Holder diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp index a1d9ee8b7c28..932917362b20 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.cpp @@ -29,6 +29,7 @@ #include #include #include +#include using namespace DB; @@ -56,7 +57,7 @@ void local_engine::PartitionWriter::write(const PartitionInfo& partition_info, D if (buffer.size() >= shuffle_writer->options.split_size) { Block block = buffer.releaseColumns(); - auto bytes = block.bytes(); + auto bytes = block.allocatedBytes(); total_partition_buffer_size += bytes; shuffle_writer->split_result.raw_partition_length[i] += bytes; partition_buffer[i].addBlock(block); @@ -73,7 +74,7 @@ void LocalPartitionWriter::evictPartitions(bool for_memory_spill) WriteBufferFromFile output(file, shuffle_writer->options.io_buffer_size); auto codec = DB::CompressionCodecFactory::instance().get(boost::to_upper_copy(shuffle_writer->options.compress_method), {}); CompressedWriteBuffer compressed_output(output, codec, shuffle_writer->options.io_buffer_size); - NativeWriter writer(compressed_output, 0, shuffle_writer->output_header); + NativeWriter writer(compressed_output, shuffle_writer->output_header); SpillInfo info; info.spilled_file = file; size_t partition_id = 0; @@ -122,7 +123,7 @@ std::vector LocalPartitionWriter::mergeSpills(WriteBuffer& data_file) { auto codec = DB::CompressionCodecFactory::instance().get(boost::to_upper_copy(shuffle_writer->options.compress_method), {}); CompressedWriteBuffer compressed_output(data_file, codec, shuffle_writer->options.io_buffer_size); - NativeWriter writer(compressed_output, 0, shuffle_writer->output_header); + NativeWriter writer(compressed_output, shuffle_writer->output_header); std::vector partition_length; partition_length.resize(shuffle_writer->options.partition_nums, 0); @@ -130,7 +131,8 @@ std::vector LocalPartitionWriter::mergeSpills(WriteBuffer& data_file) spill_inputs.reserve(spill_infos.size()); for (const auto & spill : spill_infos) { - spill_inputs.emplace_back(std::make_shared(spill.spilled_file, shuffle_writer->options.io_buffer_size)); + // only use readBig + spill_inputs.emplace_back(std::make_shared(spill.spilled_file, 0)); } Stopwatch write_time_watch; @@ -177,8 +179,8 @@ std::vector LocalPartitionWriter::mergeSpills(WriteBuffer& data_file) } return partition_length; } -LocalPartitionWriter::LocalPartitionWriter(CachedShuffleWriter * shuffle_writer) - : PartitionWriter(shuffle_writer) +LocalPartitionWriter::LocalPartitionWriter(CachedShuffleWriter * shuffle_writer_) + : PartitionWriter(shuffle_writer_) { } String LocalPartitionWriter::getNextSpillFile() @@ -229,7 +231,7 @@ void CelebornPartitionWriter::evictPartitions(bool for_memory_spill) WriteBufferFromOwnString output; auto codec = DB::CompressionCodecFactory::instance().get(boost::to_upper_copy(shuffle_writer->options.compress_method), {}); CompressedWriteBuffer compressed_output(output, codec, shuffle_writer->options.io_buffer_size); - NativeWriter writer(compressed_output, 0, shuffle_writer->output_header); + NativeWriter writer(compressed_output, shuffle_writer->output_header); size_t raw_size = partition.spill(writer); compressed_output.sync(); Stopwatch push_time_watch; @@ -286,9 +288,8 @@ void CelebornPartitionWriter::stop() void Partition::addBlock(DB::Block & block) { - std::unique_lock lock(mtx, std::try_to_lock); - if (lock.owns_lock()) - blocks.emplace_back(std::move(block)); + std::unique_lock lock(mtx); + blocks.emplace_back(std::move(block)); } bool Partition::empty() const @@ -303,16 +304,19 @@ void Partition::clear() blocks.clear(); } -size_t Partition::spill(DB::NativeWriter & writer) +size_t Partition::spill(NativeWriter & writer) { std::unique_lock lock(mtx, std::try_to_lock); if (lock.owns_lock()) { size_t raw_size = 0; - for (const auto & block : blocks) + while (!blocks.empty()) { + auto & block = blocks.back(); raw_size += writer.write(block); + blocks.pop_back(); } + blocks.clear(); return raw_size; } else diff --git a/cpp-ch/local-engine/Shuffle/PartitionWriter.h b/cpp-ch/local-engine/Shuffle/PartitionWriter.h index a0d83c194b05..0a457e39415e 100644 --- a/cpp-ch/local-engine/Shuffle/PartitionWriter.h +++ b/cpp-ch/local-engine/Shuffle/PartitionWriter.h @@ -48,7 +48,7 @@ class Partition void addBlock(DB::Block & block); bool empty() const; void clear(); - size_t spill(DB::NativeWriter & writer); + size_t spill(NativeWriter & writer); private: std::vector blocks; diff --git a/cpp-ch/local-engine/Shuffle/ShuffleReader.cpp b/cpp-ch/local-engine/Shuffle/ShuffleReader.cpp index 32c9e4cecf08..50165ca6661a 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleReader.cpp +++ b/cpp-ch/local-engine/Shuffle/ShuffleReader.cpp @@ -37,11 +37,11 @@ local_engine::ShuffleReader::ShuffleReader(std::unique_ptr in_, bool { compressed_in = std::make_unique(*in); configureCompressedReadBuffer(static_cast(*compressed_in)); - input_stream = std::make_unique(*compressed_in, 0); + input_stream = std::make_unique(*compressed_in); } else { - input_stream = std::make_unique(*in, 0); + input_stream = std::make_unique(*in); } } Block * local_engine::ShuffleReader::read() diff --git a/cpp-ch/local-engine/Shuffle/ShuffleReader.h b/cpp-ch/local-engine/Shuffle/ShuffleReader.h index fccc2b0e5755..082e75a26ca6 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleReader.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleReader.h @@ -19,6 +19,7 @@ #include #include #include +#include namespace DB { @@ -42,7 +43,7 @@ class ShuffleReader : BlockIterator private: std::unique_ptr in; std::unique_ptr compressed_in; - std::unique_ptr input_stream; + std::unique_ptr input_stream; DB::Block header; }; diff --git a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp b/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp index 2f7d14410284..e45b9d32218e 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp +++ b/cpp-ch/local-engine/Shuffle/ShuffleSplitter.cpp @@ -21,7 +21,7 @@ #include #include #include -#include +#include #include #include #include @@ -39,7 +39,12 @@ void ShuffleSplitter::split(DB::Block & block) { return; } + initOutputIfNeeded(block); computeAndCountPartitionId(block); + Stopwatch split_time_watch; + split_time_watch.start(); + block = convertAggregateStateInBlock(block); + split_result.total_split_time += split_time_watch.elapsedNanoseconds(); splitBlockByPartition(block); } SplitResult ShuffleSplitter::stop() @@ -70,12 +75,12 @@ SplitResult ShuffleSplitter::stop() stopped = true; return split_result; } -void ShuffleSplitter::splitBlockByPartition(DB::Block & block) + +void ShuffleSplitter::initOutputIfNeeded(Block & block) { - Stopwatch split_time_watch; - split_time_watch.start(); - if (!output_header.columns()) [[unlikely]] + if (output_header.columns() == 0) [[unlikely]] { + output_header = block.cloneEmpty(); if (output_columns_indicies.empty()) { output_header = block.cloneEmpty(); @@ -86,7 +91,7 @@ void ShuffleSplitter::splitBlockByPartition(DB::Block & block) } else { - DB::ColumnsWithTypeAndName cols; + ColumnsWithTypeAndName cols; for (const auto & index : output_columns_indicies) { cols.push_back(block.getByPosition(index)); @@ -94,6 +99,12 @@ void ShuffleSplitter::splitBlockByPartition(DB::Block & block) output_header = DB::Block(cols); } } +} + +void ShuffleSplitter::splitBlockByPartition(DB::Block & block) +{ + Stopwatch split_time_watch; + split_time_watch.start(); DB::Block out_block; for (size_t col = 0; col < output_header.columns(); ++col) { @@ -148,7 +159,7 @@ void ShuffleSplitter::spillPartition(size_t partition_id) { partition_write_buffers[partition_id] = getPartitionWriteBuffer(partition_id); partition_outputs[partition_id] - = std::make_unique(*partition_write_buffers[partition_id], 0, partition_buffer[partition_id].getHeader()); + = std::make_unique(*partition_write_buffers[partition_id], output_header); } DB::Block result = partition_buffer[partition_id].releaseColumns(); if (result.rows() > 0) @@ -383,7 +394,7 @@ HashSplitter::HashSplitter(SplitOptions options_) : ShuffleSplitter(std::move(op output_columns_indicies.push_back(std::stoi(*iter)); } - selector_builder = std::make_unique(options.partition_nums, hash_fields, "cityHash64"); + selector_builder = std::make_unique(options.partition_nums, hash_fields, options_.hash_algorithm); } std::unique_ptr HashSplitter::create(SplitOptions && options_) { diff --git a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h b/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h index 49fa967fc03e..aad53508b81b 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleSplitter.h @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include #include @@ -47,7 +47,7 @@ struct SplitOptions std::string compress_method = "zstd"; int compress_level; size_t spill_threshold = 300 * 1024 * 1024; - + std::string hash_algorithm; }; class ColumnsBuffer @@ -101,6 +101,7 @@ class ShuffleSplitter : public ShuffleWriterBase private: void init(); + void initOutputIfNeeded(DB::Block & block); void splitBlockByPartition(DB::Block & block); void spillPartition(size_t partition_id); std::string getPartitionTempFile(size_t partition_id); @@ -111,7 +112,7 @@ class ShuffleSplitter : public ShuffleWriterBase bool stopped = false; PartitionInfo partition_info; std::vector partition_buffer; - std::vector> partition_outputs; + std::vector> partition_outputs; std::vector> partition_write_buffers; std::vector> partition_cached_write_buffers; std::vector compressed_buffers; diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp index 8fdbac37fd60..dddf0b895fdf 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.cpp @@ -41,11 +41,11 @@ void ShuffleWriter::write(const Block & block) { if (compression_enable) { - native_writer = std::make_unique(*compressed_out, 0, block.cloneEmpty()); + native_writer = std::make_unique(*compressed_out, block.cloneEmpty()); } else { - native_writer = std::make_unique(*write_buffer, 0, block.cloneEmpty()); + native_writer = std::make_unique(*write_buffer, block.cloneEmpty()); } } if (block.rows() > 0) diff --git a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h index 459bf4e93ad7..98f67d1ccadb 100644 --- a/cpp-ch/local-engine/Shuffle/ShuffleWriter.h +++ b/cpp-ch/local-engine/Shuffle/ShuffleWriter.h @@ -16,7 +16,7 @@ */ #pragma once #include -#include +#include namespace local_engine { @@ -32,7 +32,7 @@ class ShuffleWriter private: std::unique_ptr compressed_out; std::unique_ptr write_buffer; - std::unique_ptr native_writer; + std::unique_ptr native_writer; bool compression_enable; }; } diff --git a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp b/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp index e498dc0c325d..c767a6fd3e7a 100644 --- a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp +++ b/cpp-ch/local-engine/Storages/CustomStorageMergeTree.cpp @@ -92,5 +92,8 @@ std::unique_ptr CustomStorageMergeTree::getDefaultSettings() { throw std::runtime_error("not implement"); } - +std::map CustomStorageMergeTree::getUnfinishedMutationCommands() const +{ + throw std::runtime_error("not implement"); +} } diff --git a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h b/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h index e61a54eef142..4ac989d21206 100644 --- a/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h +++ b/cpp-ch/local-engine/Storages/CustomStorageMergeTree.h @@ -45,6 +45,7 @@ class CustomStorageMergeTree final : public MergeTreeData std::string getName() const override; std::vector getMutationsStatus() const override; bool scheduleDataProcessingJob(BackgroundJobsAssignee & executor) override; + std::map getUnfinishedMutationCommands() const override; MergeTreeDataWriter writer; MergeTreeDataSelectExecutor reader; @@ -64,7 +65,6 @@ class CustomStorageMergeTree final : public MergeTreeData void replacePartitionFrom(const StoragePtr & source_table, const ASTPtr & partition, bool replace, ContextPtr context) override; void movePartitionToTable(const StoragePtr & dest_table, const ASTPtr & partition, ContextPtr context) override; bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; - size_t getNumberOfUnfinishedMutations() const override { return 0; } std::map getAlterMutationCommandsForPart(const DataPartPtr & /*part*/) const override { return {}; } void attachRestoredParts(MutableDataPartsVector && /*parts*/) override { throw std::runtime_error("not implement"); } }; diff --git a/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp new file mode 100644 index 000000000000..84c32f4565f7 --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.cpp @@ -0,0 +1,143 @@ +/* + * 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 "AggregateSerializationUtils.h" +#include + +#include +#include +#include +#include +#include +#include + + +using namespace DB; + +namespace local_engine +{ + +bool isFixedSizeStateAggregateFunction(const String& name) +{ + static const std::set function_set = {"min", "max", "sum", "count", "avg"}; + return function_set.contains(name); +} + +bool isFixedSizeArguments(const DataTypes& data_types) +{ + return removeNullable(data_types.front())->isValueRepresentedByNumber(); +} + +bool isFixedSizeAggregateFunction(const DB::AggregateFunctionPtr& function) +{ + return isFixedSizeStateAggregateFunction(function->getName()) && isFixedSizeArguments(function->getArgumentTypes()); +} + +DB::ColumnWithTypeAndName convertAggregateStateToFixedString(const DB::ColumnWithTypeAndName& col) +{ + const auto *aggregate_col = checkAndGetColumn(*col.column); + if (!aggregate_col) + { + return col; + } + // only support known fixed size aggregate function + if (!isFixedSizeAggregateFunction(aggregate_col->getAggregateFunction())) + { + return col; + } + size_t state_size = aggregate_col->getAggregateFunction()->sizeOfData(); + auto res_type = std::make_shared(state_size); + auto res_col = res_type->createColumn(); + PaddedPODArray & column_chars_t = assert_cast(*res_col).getChars(); + column_chars_t.reserve(aggregate_col->size() * state_size); + for (const auto & item : aggregate_col->getData()) + { + column_chars_t.insert_assume_reserved(item, item + state_size); + } + return DB::ColumnWithTypeAndName(std::move(res_col), res_type, col.name); +} + +DB::ColumnWithTypeAndName convertAggregateStateToString(const DB::ColumnWithTypeAndName& col) +{ + const auto *aggregate_col = checkAndGetColumn(*col.column); + if (!aggregate_col) + { + return col; + } + auto res_type = std::make_shared(); + auto res_col = res_type->createColumn(); + PaddedPODArray & column_chars = assert_cast(*res_col).getChars(); + IColumn::Offsets & column_offsets = assert_cast(*res_col).getOffsets(); + auto value_writer = WriteBufferFromVector>(column_chars); + column_offsets.reserve(aggregate_col->size()); + for (const auto & item : aggregate_col->getData()) + { + aggregate_col->getAggregateFunction()->serialize(item, value_writer); + writeChar('\0', value_writer); + column_offsets.emplace_back(value_writer.count()); + } + return DB::ColumnWithTypeAndName(std::move(res_col), res_type, col.name); +} + +DB::ColumnWithTypeAndName convertFixedStringToAggregateState(const DB::ColumnWithTypeAndName & col, const DB::DataTypePtr & type) +{ + chassert(WhichDataType(type).isAggregateFunction()); + auto res_col = type->createColumn(); + const auto * agg_type = checkAndGetDataType(type.get()); + ColumnAggregateFunction & real_column = typeid_cast(*res_col); + auto & arena = real_column.createOrGetArena(); + ColumnAggregateFunction::Container & vec = real_column.getData(); + vec.reserve(col.column->size()); + auto agg_function = agg_type->getFunction(); + size_t size_of_state = agg_function->sizeOfData(); + size_t align_of_state = agg_function->alignOfData(); + + for (size_t i = 0; i < col.column->size(); ++i) + { + AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); + + agg_function->create(place); + + auto value = col.column->getDataAt(i); + memcpy(place, value.data, value.size); + + vec.push_back(place); + } + return DB::ColumnWithTypeAndName(std::move(res_col), type, col.name); +} +DB::Block convertAggregateStateInBlock(DB::Block& block) +{ + ColumnsWithTypeAndName columns; + columns.reserve(block.columns()); + for (const auto & item : block.getColumnsWithTypeAndName()) + { + if (WhichDataType(item.type).isAggregateFunction()) + { + const auto *aggregate_col = checkAndGetColumn(*item.column); + if (isFixedSizeAggregateFunction(aggregate_col->getAggregateFunction())) + columns.emplace_back(convertAggregateStateToFixedString(item)); + else + columns.emplace_back(convertAggregateStateToString(item)); + } + else + { + columns.emplace_back(item); + } + } + return columns; +} +} + diff --git a/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.h b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.h new file mode 100644 index 000000000000..6536982ef572 --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/AggregateSerializationUtils.h @@ -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. + */ +#pragma once +#include +#include +#include + +namespace local_engine { + +bool isFixedSizeAggregateFunction(const DB::AggregateFunctionPtr & function); + +DB::Block convertAggregateStateInBlock(DB::Block& block); + +DB::ColumnWithTypeAndName convertAggregateStateToFixedString(const DB::ColumnWithTypeAndName & col); + +DB::ColumnWithTypeAndName convertAggregateStateToString(const DB::ColumnWithTypeAndName & col); + +DB::ColumnWithTypeAndName convertFixedStringToAggregateState(const DB::ColumnWithTypeAndName & col, const DB::DataTypePtr & type); + +} + diff --git a/cpp-ch/local-engine/Storages/IO/NativeReader.cpp b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp new file mode 100644 index 000000000000..12955579c624 --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/NativeReader.cpp @@ -0,0 +1,179 @@ +/* + * 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 "NativeReader.h" + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int INCORRECT_INDEX; + extern const int LOGICAL_ERROR; + extern const int CANNOT_READ_ALL_DATA; + extern const int INCORRECT_DATA; + extern const int TOO_LARGE_ARRAY_SIZE; +} +} + +using namespace DB; + +namespace local_engine +{ +void NativeReader::readData(const ISerialization & serialization, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +{ + ISerialization::DeserializeBinaryBulkSettings settings; + settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; + settings.avg_value_size_hint = avg_value_size_hint; + settings.position_independent_encoding = false; + settings.native_format = true; + + ISerialization::DeserializeBinaryBulkStatePtr state; + + serialization.deserializeBinaryBulkStatePrefix(settings, state); + serialization.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state, nullptr); + + if (column->size() != rows) + throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, + "Cannot read all data in NativeReader. Rows read: {}. Rows expected: {}", column->size(), rows); +} + +template +void NativeReader::readAggData(const DB::DataTypeAggregateFunction & data_type, DB::ColumnPtr & column, DB::ReadBuffer & istr, size_t rows) +{ + ColumnAggregateFunction & real_column = typeid_cast(*column->assumeMutable()); + auto & arena = real_column.createOrGetArena(); + ColumnAggregateFunction::Container & vec = real_column.getData(); + + vec.reserve(rows); + auto agg_function = data_type.getFunction(); + size_t size_of_state = agg_function->sizeOfData(); + size_t align_of_state = agg_function->alignOfData(); + + for (size_t i = 0; i < rows; ++i) + { + AggregateDataPtr place = arena.alignedAlloc(size_of_state, align_of_state); + agg_function->create(place); + if constexpr (FIXED) + { + auto n = istr.read(place, size_of_state); + chassert(n == size_of_state); + } + else + { + agg_function->deserialize(place, istr, std::nullopt, &arena); + istr.ignore(); + } + + vec.push_back(place); + } +} + + +Block NativeReader::getHeader() const +{ + return header; +} + +Block NativeReader::read() +{ + Block res; + + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); + + if (istr.eof()) + { + return res; + } + + /// Dimensions + size_t columns = 0; + size_t rows = 0; + + readVarUInt(columns, istr); + readVarUInt(rows, istr); + + if (columns > 1'000'000uz) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Suspiciously many columns in Native format: {}", columns); + if (rows > 1'000'000'000'000uz) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Suspiciously many rows in Native format: {}", rows); + + if (columns == 0 && !header && rows != 0) + throw Exception(ErrorCodes::INCORRECT_DATA, "Zero columns but {} rows in Native format.", rows); + + for (size_t i = 0; i < columns; ++i) + { + ColumnWithTypeAndName column; + + column.name = "col_" + std::to_string(i); + + /// Type + String type_name; + readBinary(type_name, istr); + bool agg_opt_column = false; + String real_type_name = type_name; + if (type_name.ends_with(NativeWriter::AGG_STATE_SUFFIX)) + { + agg_opt_column = true; + real_type_name = type_name.substr(0, type_name.length() - NativeWriter::AGG_STATE_SUFFIX.length()); + } + column.type = data_type_factory.get(real_type_name); + bool is_agg_state_type = WhichDataType(column.type).isAggregateFunction(); + SerializationPtr serialization = column.type->getDefaultSerialization(); + + /// Data + ColumnPtr read_column = column.type->createColumn(*serialization); + + double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; + if (rows) /// If no rows, nothing to read. + { + if (is_agg_state_type && agg_opt_column) + { + const DataTypeAggregateFunction * agg_type = checkAndGetDataType(column.type.get()); + bool fixed = isFixedSizeAggregateFunction(agg_type->getFunction()); + if (fixed) + { + readAggData(*agg_type, read_column, istr, rows); + } + else + { + readAggData(*agg_type, read_column, istr, rows); + } + } + else + { + readData(*serialization, read_column, istr, rows, avg_value_size_hint); + } + } + column.column = std::move(read_column); + + res.insert(std::move(column)); + } + + if (res.rows() != rows) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Row count mismatch after deserialization, got: {}, expected: {}", res.rows(), rows); + + return res; +} + +} diff --git a/cpp-ch/local-engine/Storages/IO/NativeReader.h b/cpp-ch/local-engine/Storages/IO/NativeReader.h new file mode 100644 index 000000000000..d065fce347d4 --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/NativeReader.h @@ -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. + */ +#pragma once + +#include +#include +#include + +namespace local_engine +{ + +class NativeReader +{ +public: + NativeReader(DB::ReadBuffer & istr_) : istr(istr_) {} + + static void readData(const DB::ISerialization & serialization, DB::ColumnPtr & column, DB::ReadBuffer & istr, size_t rows, double avg_value_size_hint); + template + static void readAggData(const DB::DataTypeAggregateFunction & data_type, DB::ColumnPtr & column, DB::ReadBuffer & istr, size_t rows); + + DB::Block getHeader() const; + + DB::Block read(); + +private: + DB::ReadBuffer & istr; + DB::Block header; + + DB::PODArray avg_value_size_hints; + + void updateAvgValueSizeHints(const DB::Block & block); +}; + +} diff --git a/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp new file mode 100644 index 000000000000..39a0cb7b579b --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/NativeWriter.cpp @@ -0,0 +1,111 @@ +/* + * 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 "NativeWriter.h" +#include +#include +#include +#include +#include +#include +#include +#include + + +using namespace DB; + +namespace local_engine +{ + +const String NativeWriter::AGG_STATE_SUFFIX= "#optagg"; +void NativeWriter::flush() +{ + ostr.next(); +} + +static void writeData(const ISerialization & serialization, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +{ + /** If there are columns-constants - then we materialize them. + * (Since the data type does not know how to serialize / deserialize constants.) + */ + ColumnPtr full_column = column->convertToFullColumnIfConst(); + + ISerialization::SerializeBinaryBulkSettings settings; + settings.getter = [&ostr](ISerialization::SubstreamPath) -> WriteBuffer * { return &ostr; }; + settings.position_independent_encoding = false; + settings.low_cardinality_max_dictionary_size = 0; + + ISerialization::SerializeBinaryBulkStatePtr state; + serialization.serializeBinaryBulkStatePrefix(*full_column, settings, state); + serialization.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); + serialization.serializeBinaryBulkStateSuffix(settings, state); +} + +size_t NativeWriter::write(const DB::Block & block) +{ + size_t written_before = ostr.count(); + + block.checkNumberOfRows(); + + /// Dimensions + size_t columns = block.columns(); + size_t rows = block.rows(); + + writeVarUInt(columns, ostr); + writeVarUInt(rows, ostr); + + for (size_t i = 0; i < columns; ++i) + { + auto column = block.safeGetByPosition(i); + /// agg state will convert to fixedString, need write actual agg state type + auto original_type = header.safeGetByPosition(i).type; + /// Type + String type_name = original_type->getName(); + bool is_agg_opt = WhichDataType(original_type).isAggregateFunction() + && header.safeGetByPosition(i).column->getDataType() != block.safeGetByPosition(i).column->getDataType(); + if (is_agg_opt) + { + writeStringBinary(type_name + AGG_STATE_SUFFIX, ostr); + } + else + { + writeStringBinary(type_name, ostr); + } + + SerializationPtr serialization = column.type->getDefaultSerialization(); + column.column = recursiveRemoveSparse(column.column); + /// Data + if (rows) /// Zero items of data is always represented as zero number of bytes. + { + const auto * agg_type = checkAndGetDataType(original_type.get()); + if (is_agg_opt && agg_type && !isFixedSizeAggregateFunction(agg_type->getFunction())) + { + const auto * str_col = static_cast(column.column.get()); + const PaddedPODArray & column_chars = str_col->getChars(); + ostr.write(column_chars.raw_data(), str_col->getOffsets().back()); + } + else + { + writeData(*serialization, column.column, ostr, 0, 0); + } + } + } + + size_t written_after = ostr.count(); + size_t written_size = written_after - written_before; + return written_size; +} +} diff --git a/cpp-ch/local-engine/Storages/IO/NativeWriter.h b/cpp-ch/local-engine/Storages/IO/NativeWriter.h new file mode 100644 index 000000000000..a958f4484dce --- /dev/null +++ b/cpp-ch/local-engine/Storages/IO/NativeWriter.h @@ -0,0 +1,50 @@ +/* + * 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 DB +{ +class WriteBuffer; +class CompressedWriteBuffer; +} + +namespace local_engine +{ + +class NativeWriter +{ +public: + static const String AGG_STATE_SUFFIX; + NativeWriter( + DB::WriteBuffer & ostr_, const DB::Block & header_): ostr(ostr_), header(header_) + {} + + DB::Block getHeader() const { return header; } + /// Returns the number of bytes written. + size_t write(const DB::Block & block); + void flush(); + + +private: + DB::WriteBuffer & ostr; + DB::Block header; +}; +} diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelNumberReader.h b/cpp-ch/local-engine/Storages/Serializations/ExcelNumberReader.h index 3e51d6f0e66a..38b8abd1429e 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelNumberReader.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelNumberReader.h @@ -333,6 +333,7 @@ inline bool readExcelFloatTextFastImpl(T & x, DB::ReadBuffer & in, bool has_quot template bool readExcelIntTextImpl(T & x, DB::ReadBuffer & buf, bool has_quote, const DB::FormatSettings & settings) { + bool number_force = settings.try_infer_integers==1; const UInt8 MAX_HEAD_SKIP = 2; const UInt8 MAX_TAIL_SKIP = 2; UInt8 head_skip=0; @@ -400,7 +401,10 @@ bool readExcelIntTextImpl(T & x, DB::ReadBuffer & buf, bool has_quote, const DB: { if (!(*buf.position() >= '0' && *buf.position() <= '9')) { - break; + if (number_force) + break; + else + return false; } else { @@ -448,7 +452,7 @@ bool readExcelIntTextImpl(T & x, DB::ReadBuffer & buf, bool has_quote, const DB: { continue; } - else if (has_number && !(*buf.position() >= '0' && *buf.position() <= '9')) // process suffix + else if (has_number && !(*buf.position() >= '0' && *buf.position() <= '9') && number_force) // process suffix { while (!buf.eof()) { @@ -467,7 +471,7 @@ bool readExcelIntTextImpl(T & x, DB::ReadBuffer & buf, bool has_quote, const DB: } break; } - else if (!has_number && !(*buf.position() >= '0' && *buf.position() <= '9')) // process prefix + else if (!has_number && !(*buf.position() >= '0' && *buf.position() <= '9') && number_force) // process prefix { if(*buf.position() == settings.csv.delimiter || *buf.position() == '\n' || *buf.position() == '\r') { diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.cpp b/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.cpp index 7966878953b5..c44ed502ae9f 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.cpp +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.cpp @@ -54,79 +54,134 @@ inline void appendToStringOrVector(PaddedPODArray & s, ReadBuffer & rb, c } template -void readExcelCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings, const String & escape_value) +void readExcelCSVQuoteString(Vector & s, ReadBuffer & buf, const char delimiter, const String & escape_value, const char & quote) { - /// Empty string - if (buf.eof()) - return; + if constexpr (include_quotes) + s.push_back(quote); - const char delimiter = settings.delimiter; - const char maybe_quote = *buf.position(); - const String & custom_delimiter = settings.custom_delimiter; - - /// Emptiness and not even in quotation marks. - if (custom_delimiter.empty() && maybe_quote == delimiter) - return; - - if ((settings.allow_single_quotes && maybe_quote == '\'') || (settings.allow_double_quotes && maybe_quote == '"')) + /// The quoted case. We are looking for the next quotation mark. + while (!buf.eof()) { - if constexpr (include_quotes) - s.push_back(maybe_quote); - - ++buf.position(); + char * next_pos = buf.position(); - /// The quoted case. We are looking for the next quotation mark. - while (!buf.eof()) + [&]() { - char * next_pos = reinterpret_cast(memchr(buf.position(), maybe_quote, buf.buffer().end() - buf.position())); - - if (nullptr == next_pos) - next_pos = buf.buffer().end(); +#ifdef __SSE2__ + auto qe = _mm_set1_epi8(quote); + for (; next_pos + 15 < buf.buffer().end(); next_pos += 16) + { + __m128i bytes = _mm_loadu_si128(reinterpret_cast(next_pos)); + auto eq = _mm_cmpeq_epi8(bytes, qe); + if (!escape_value.empty()) + { + eq = _mm_or_si128(eq, _mm_cmpeq_epi8(bytes, _mm_set1_epi8(escape_value[0]))); + } - bool escape = false; - if (escape_value != "" && next_pos + 1 <= buf.buffer().end() && *(next_pos - 1) == escape_value[0]) + uint16_t bit_mask = _mm_movemask_epi8(eq); + if (bit_mask) + { + next_pos += std::countr_zero(bit_mask); + return; + } + } +//#elif defined(__aarch64__) && defined(__ARM_NEON) +// auto rc = vdupq_n_u8('\r'); +// auto nc = vdupq_n_u8('\n'); +// auto dc = vdupq_n_u8(delimiter); +// /// Returns a 64 bit mask of nibbles (4 bits for each byte). +// auto get_nibble_mask = [](uint8x16_t input) -> uint64_t +// { return vget_lane_u64(vreinterpret_u64_u8(vshrn_n_u16(vreinterpretq_u16_u8(input), 4)), 0); }; +// for (; next_pos + 15 < buf.buffer().end(); next_pos += 16) +// { +// uint8x16_t bytes = vld1q_u8(reinterpret_cast(next_pos)); +// auto eq = vorrq_u8(vorrq_u8(vceqq_u8(bytes, rc), vceqq_u8(bytes, nc)), vceqq_u8(bytes, dc)); +// uint64_t bit_mask = get_nibble_mask(eq); +// if (bit_mask) +// { +// next_pos += std::countr_zero(bit_mask) >> 2; +// return; +// } +// } +#endif + while (next_pos < buf.buffer().end() && *next_pos != quote) { - /// if has escape, back 1 position - escape = true; - --next_pos; + if (!escape_value.empty() && *next_pos == escape_value[0]) + break; + + ++next_pos; } + }(); + if (buf.position() != next_pos) + { appendToStringOrVector(s, buf, next_pos); + } - if (escape) + if (!escape_value.empty() && escape_value[0] == *next_pos) + { + next_pos++; + + if (next_pos != buf.buffer().end()) { - /// skip escape char - ++next_pos; - /// add maybe_quote - s.push_back(*next_pos); - ++next_pos; + if (*next_pos == quote || *next_pos == escape_value[0]) + s.push_back(*next_pos); + else + { + s.push_back(escape_value[0]); + s.push_back(*next_pos); + } + next_pos++; buf.position() = next_pos; continue; } + } - buf.position() = next_pos; + buf.position() = next_pos; + if (!buf.hasPendingData()) + continue; - if (!buf.hasPendingData()) + if (!buf.eof()) + { + auto end_char = *buf.position(); + if (end_char == quote) + buf.position()++; + + if (!buf.eof() && *buf.position() != delimiter && *buf.position() != '\r' && *buf.position() != '\n') + { + s.push_back(end_char); continue; + } + } + + if constexpr (include_quotes) + s.push_back(quote); - if constexpr (include_quotes) - s.push_back(maybe_quote); + return; + } +} - /// Now there is a quotation mark under the cursor. Is there any following? - ++buf.position(); +template +void readExcelCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings, const String & escape_value) +{ + /// Empty string + if (buf.eof()) + return; - if (buf.eof()) - return; + const char delimiter = settings.delimiter; + const char maybe_quote = *buf.position(); + const String & custom_delimiter = settings.custom_delimiter; - if (*buf.position() == maybe_quote) - { - s.push_back(maybe_quote); - ++buf.position(); - continue; - } + /// Emptiness and not even in quotation marks. + if (custom_delimiter.empty() && maybe_quote == delimiter) + return; - return; - } + if ((settings.allow_single_quotes && maybe_quote == '\'') || (settings.allow_double_quotes && maybe_quote == '"')) + { + ++buf.position(); + if (!buf.eof() && *buf.position() == '{' && *(buf.position() + 1) == maybe_quote) + readExcelCSVQuoteString(s, buf, delimiter, escape_value, maybe_quote); + else + readExcelCSVQuoteString(s, buf, delimiter, escape_value, maybe_quote); } else { diff --git a/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.h b/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.h index 76b5f918546f..7a1b3f59a3b8 100644 --- a/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.h +++ b/cpp-ch/local-engine/Storages/Serializations/ExcelStringReader.h @@ -50,8 +50,9 @@ static inline void excelRead(IColumn & column, Reader && reader) } } - template +void readExcelCSVQuoteString(Vector & s, ReadBuffer & buf, const char delimiter, const String & escape_value, const char & quote); +template void readExcelCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV & settings, const String & escape_value); diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp index 0e6b80c55c3b..d7cbcaa5fe94 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.cpp @@ -14,6 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +#include "Common/CHUtil.h" #include "ExcelTextFormatFile.h" @@ -32,7 +33,7 @@ #include #include #include - +#include namespace DB { @@ -46,17 +47,17 @@ namespace ErrorCodes namespace local_engine { -void skipErrorChars(DB::ReadBuffer & buf, bool has_quote, char maybe_quote, const DB::FormatSettings & settings) +void skipErrorChars(DB::ReadBuffer & buf, bool has_quote, char quote, String & escape, const DB::FormatSettings & settings) { - char skip_before_char = has_quote ? maybe_quote : settings.csv.delimiter; - - /// skip all chars before quote/delimiter exclude line delimiter - while (!buf.eof() && *buf.position() != skip_before_char && *buf.position() != '\n' && *buf.position() != '\r') - ++buf.position(); - - /// if char is quote, skip it - if (has_quote && !buf.eof() && *buf.position() == maybe_quote) - ++buf.position(); + if (has_quote) + { + ColumnString::Chars data; + readExcelCSVQuoteString(data, buf, settings.csv.delimiter, escape, quote); + } + else + /// skip all chars before quote/delimiter exclude line delimiter + while (!buf.eof() && *buf.position() != settings.csv.delimiter && *buf.position() != '\n' && *buf.position() != '\r') + ++buf.position(); } FormatFile::InputFormatPtr ExcelTextFormatFile::createInputFormat(const DB::Block & header) @@ -101,20 +102,45 @@ DB::FormatSettings ExcelTextFormatFile::createFormatSettings() if (!file_info.text().null_value().empty()) format_settings.csv.null_representation = file_info.text().null_value(); - format_settings.csv.empty_as_default = true; + bool empty_as_null = true; + if (context->getSettings().has(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL)) + empty_as_null = context->getSettings().getString(BackendInitializerUtil::EXCEL_EMPTY_AS_NULL) == "'true'"; + + format_settings.try_infer_integers = 0; + if (!context->getSettings().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE)) + format_settings.try_infer_integers = 1; + if (context->getSettings().has(BackendInitializerUtil::EXCEL_NUMBER_FORCE) + && context->getSettings().getString(BackendInitializerUtil::EXCEL_NUMBER_FORCE) == "'true'") + format_settings.try_infer_integers = 1; + + if (format_settings.csv.null_representation.empty() || empty_as_null) + format_settings.csv.empty_as_default = true; + else + format_settings.csv.empty_as_default = false; char quote = *file_info.text().quote().data(); + if (quote == '\'') { format_settings.csv.allow_single_quotes = true; format_settings.csv.allow_double_quotes = false; } - else + else if (quote == '\"') { /// quote == '"' and default format_settings.csv.allow_single_quotes = false; format_settings.csv.allow_double_quotes = true; } + else + { + format_settings.csv.allow_single_quotes = false; + + if (context->getSettings().has(BackendInitializerUtil::EXCEL_QUOTE_STRICT) + && context->getSettings().getString(BackendInitializerUtil::EXCEL_QUOTE_STRICT) == "'true'") + format_settings.csv.allow_double_quotes = false; + else + format_settings.csv.allow_double_quotes = true; + } return format_settings; } @@ -134,7 +160,7 @@ ExcelRowInputFormat::ExcelRowInputFormat( true, false, format_settings_, - std::make_unique(*buf_, input_field_names_, format_settings_)) + std::make_unique(*buf_, input_field_names_, escape_, format_settings_)) , escape(escape_) { DB::Serializations gluten_serializations; @@ -182,8 +208,8 @@ ExcelRowInputFormat::ExcelRowInputFormat( ExcelTextFormatReader::ExcelTextFormatReader( - DB::PeekableReadBuffer & buf_, DB::Names & input_field_names_, const DB::FormatSettings & format_settings_) - : CSVFormatReader(buf_, format_settings_), input_field_names(input_field_names_) + DB::PeekableReadBuffer & buf_, DB::Names & input_field_names_, String escape_, const DB::FormatSettings & format_settings_) + : CSVFormatReader(buf_, format_settings_), input_field_names(input_field_names_), escape(escape_) { } @@ -205,7 +231,7 @@ bool ExcelTextFormatReader::readField( bool is_last_file_column, const String &) { - if (isEndOfLine()) + if (isEndOfLine() && format_settings.csv.empty_as_default) { column.insertDefault(); return false; @@ -222,7 +248,7 @@ bool ExcelTextFormatReader::readField( /// Note: Tuples are serialized in CSV as separate columns, but with empty_as_default or null_as_default /// only one empty or NULL column will be expected - if (format_settings.csv.empty_as_default && (at_delimiter || at_last_column_line_end)) + if ((at_delimiter || at_last_column_line_end) && (format_settings.csv.empty_as_default || !isStringOrFixedString(removeNullable(type)))) { /// Treat empty unquoted column value as default value, if /// specified in the settings. Tuple columns might seem @@ -263,7 +289,7 @@ bool ExcelTextFormatReader::readField( if (!isParseError(e.code())) throw; - skipErrorChars(*buf, has_quote, maybe_quote, format_settings); + skipErrorChars(*buf, has_quote, maybe_quote, escape, format_settings); column_back_func(column); column.insertDefault(); @@ -272,7 +298,7 @@ bool ExcelTextFormatReader::readField( if (column_size == column.size()) { - skipErrorChars(*buf, has_quote, maybe_quote, format_settings); + skipErrorChars(*buf, has_quote, maybe_quote, escape, format_settings); column_back_func(column); column.insertDefault(); return false; @@ -281,6 +307,13 @@ bool ExcelTextFormatReader::readField( return true; } +void ExcelTextFormatReader::skipField() +{ + skipWhitespacesAndTabs(*buf, format_settings.csv.allow_whitespace_or_tab_as_delimiter); + ColumnString::Chars data; + readExcelCSVStringInto(data, *buf, format_settings.csv, escape); +} + void ExcelTextFormatReader::preSkipNullValue() { /// null_representation is empty and value is "" or '' in spark return null @@ -346,37 +379,37 @@ void ExcelTextFormatReader::skipRowEndDelimiter() skipEndOfLine(*buf); } -void ExcelTextFormatReader::skipEndOfLine(DB::ReadBuffer & in) +void ExcelTextFormatReader::skipEndOfLine(DB::ReadBuffer & readBuffer) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) - if (*in.position() == '\n') + if (*readBuffer.position() == '\n') { - ++in.position(); - if (!in.eof() && *in.position() == '\r') - ++in.position(); + ++readBuffer.position(); + if (!readBuffer.eof() && *readBuffer.position() == '\r') + ++readBuffer.position(); } - else if (*in.position() == '\r') + else if (*readBuffer.position() == '\r') { - ++in.position(); - if (!in.eof() && *in.position() == '\n') - ++in.position(); + ++readBuffer.position(); + if (!readBuffer.eof() && *readBuffer.position() == '\n') + ++readBuffer.position(); /// Different with CH master: /// removed \r check } - else if (!in.eof()) + else if (!readBuffer.eof()) throw DB::Exception(DB::ErrorCodes::INCORRECT_DATA, "Expected end of line"); } -inline void ExcelTextFormatReader::skipWhitespacesAndTabs(ReadBuffer & in, bool allow_whitespace_or_tab_as_delimiter) +inline void ExcelTextFormatReader::skipWhitespacesAndTabs(ReadBuffer & readBuffer, bool allow_whitespace_or_tab_as_delimiter) { if (allow_whitespace_or_tab_as_delimiter) { return; } - /// Skip `whitespace` symbols allowed in CSV. - while (!in.eof() && (*in.position() == ' ' || *in.position() == '\t')) - ++in.position(); + /// Skip `whitespace` symbols allowed readBuffer CSV. + while (!readBuffer.eof() && (*readBuffer.position() == ' ' || *readBuffer.position() == '\t')) + ++readBuffer.position(); } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h index b6046a74369e..4ccd969d4752 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ExcelTextFormatFile.h @@ -41,7 +41,7 @@ class ExcelTextFormatFile : public FormatFile FormatFile::InputFormatPtr createInputFormat(const DB::Block & header) override; bool supportSplit() const override { return true; } - DB::String getFileFormat() const override { return "exceltext"; } + DB::String getFileFormat() const override { return "ExcelText"; } private: DB::FormatSettings createFormatSettings(); @@ -68,21 +68,24 @@ class ExcelRowInputFormat final : public DB::CSVRowInputFormat class ExcelTextFormatReader final : public DB::CSVFormatReader { public: - ExcelTextFormatReader(DB::PeekableReadBuffer & buf_, DB::Names & input_field_names_, const DB::FormatSettings & format_settings_); + ExcelTextFormatReader( + DB::PeekableReadBuffer & buf_, DB::Names & input_field_names_, String escape_, const DB::FormatSettings & format_settings_); std::vector readNames() override; std::vector readTypes() override; void skipFieldDelimiter() override; void skipRowEndDelimiter() override; bool readField(DB::IColumn & column, const DB::DataTypePtr & type, const DB::SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + void skipField(size_t /*file_column*/) override { skipField(); } + void skipField(); private: void preSkipNullValue(); bool isEndOfLine(); - static void skipEndOfLine(DB::ReadBuffer & in); - static void skipWhitespacesAndTabs(DB::ReadBuffer & in, bool allow_whitespace_or_tab_as_delimiter); - + static void skipEndOfLine(DB::ReadBuffer & readBuffer); + static void skipWhitespacesAndTabs(DB::ReadBuffer & readBuffer, bool allow_whitespace_or_tab_as_delimiter); std::vector input_field_names; + String escape; }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h index 844f34fd1766..2f8b4dc16a71 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/FormatFile.h @@ -81,17 +81,16 @@ class FormatFile virtual size_t getStartOffset() const { return file_info.start(); } virtual size_t getLength() const { return file_info.length(); } - - void setFilters(std::vector filters_) { filters = std::move(filters_); } virtual String getFileFormat() const = 0; - + protected: DB::ContextPtr context; substrait::ReadRel::LocalFiles::FileOrFiles file_info; ReadBufferBuilderPtr read_buffer_builder; std::vector partition_keys; std::map partition_values; - std::vector filters; + // std::optional filter; + std::shared_ptr key_condition; }; using FormatFilePtr = std::shared_ptr; using FormatFiles = std::vector; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.h index 77bda5ec3cec..0b89ffc31aef 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/JSONFormatFile.h @@ -29,7 +29,7 @@ class JSONFormatFile : public FormatFile bool supportSplit() const override { return true; } FormatFile::InputFormatPtr createInputFormat(const DB::Block & header) override; - - DB::String getFileFormat() const override { return "json"; } + + DB::String getFileFormat() const override { return "JSONEachRow"; } }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.h index 40d34b60eeec..940656d14b86 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ORCFormatFile.h @@ -49,7 +49,8 @@ class ORCFormatFile : public FormatFile std::optional getTotalRows() override; bool supportSplit() const override { return true; } - DB::String getFileFormat() const override { return "orc"; } + + DB::String getFileFormat() const override { return "ORC"; } private: mutable std::mutex mutex; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp index 4fb2a7ea8584..bdb7d1915e3f 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.cpp @@ -21,6 +21,7 @@ #include #include #include + #include #include #include @@ -49,12 +50,11 @@ ParquetFormatFile::ParquetFormatFile( DB::ContextPtr context_, const substrait::ReadRel::LocalFiles::FileOrFiles & file_info_, ReadBufferBuilderPtr read_buffer_builder_, - bool useLocalFormat) - : FormatFile(context_, file_info_, read_buffer_builder_) - , use_local_format(useLocalFormat) - , enable_row_group_maxmin_index(file_info_.parquet().enable_row_group_maxmin_index()) + bool use_local_format_) + : FormatFile(context_, file_info_, read_buffer_builder_), use_local_format(use_local_format_) { } + FormatFile::InputFormatPtr ParquetFormatFile::createInputFormat(const DB::Block & header) { auto res = std::make_shared(); @@ -171,9 +171,6 @@ std::vector ParquetFormatFile::collectRequiredRowGroups(DB:: for (int j = 0; j < row_group_meta->num_columns(); ++j) total_bytes += row_group_meta->ColumnChunk(j)->total_compressed_size(); - if (enable_row_group_maxmin_index && !checkRowGroupIfRequired(*row_group_meta)) - continue; - UInt64 midpoint_offset = static_cast(start_offset + total_bytes / 2); /// Current row group has intersection with the required range. if (file_info.start() <= midpoint_offset && midpoint_offset < file_info.start() + file_info.length()) @@ -190,118 +187,5 @@ std::vector ParquetFormatFile::collectRequiredRowGroups(DB:: return row_group_metadatas; } -bool ParquetFormatFile::checkRowGroupIfRequired(parquet::RowGroupMetaData & meta) -{ - std::vector column_max_mins; - DB::DataTypes column_types; - const parquet::SchemaDescriptor * schema_desc = meta.schema(); - bool row_group_required = true; - for (size_t i = 0; i < filters.size(); ++i) - { - std::vector ranges; - auto iter = filters[i].keys.begin(); - - for (size_t j = 0; j < filters[i].keys.size(); ++j) - { - DB::String filter_col_key = iter->name; - DB::DataTypePtr filter_col_type = iter->type; - int column_index = schema_desc->ColumnIndex(filter_col_key); - DB::Range range = DB::Range::createWholeUniverse(); - if (column_index < 0) - ranges.emplace_back(std::move(range)); - else - { - const parquet::ColumnDescriptor * desc = schema_desc->Column(column_index); - Int32 column_type_length = desc->type_length(); - auto column_chunk_meta = meta.ColumnChunk(column_index); - if (column_chunk_meta->is_stats_set()) - range - = getColumnMaxMin(column_chunk_meta->statistics(), column_chunk_meta->type(), filter_col_type, column_type_length); - - ranges.emplace_back(std::move(range)); - } - - ++iter; - } - if (ranges.size() > 0 && !filters[i].filter.checkInHyperrectangle(ranges, filters[i].keys.getTypes()).can_be_true) - { - row_group_required = false; - break; - } - } - return row_group_required; -} - -DB::Range ParquetFormatFile::getColumnMaxMin( - std::shared_ptr statistics, - parquet::Type::type parquet_data_type, - DB::DataTypePtr data_type, - Int32 column_type_length) -{ - DB::Range maxmin = DB::Range::createWholeUniverse(); - DB::WhichDataType which_data_type(data_type); - if (parquet_data_type == parquet::Type::type::FLOAT) - { - auto float_stats = std::dynamic_pointer_cast(statistics); - maxmin = float_stats->HasMinMax() && (!float_stats->HasNullCount() || float_stats->null_count() == 0) - ? DB::Range(float_stats->min(), true, float_stats->max(), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::DOUBLE) - { - auto double_stats = std::dynamic_pointer_cast(statistics); - maxmin = double_stats->HasMinMax() && (!double_stats->HasNullCount() || double_stats->null_count() == 0) - ? DB::Range(double_stats->min(), true, double_stats->max(), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::INT32) - { - auto int32_stats = std::dynamic_pointer_cast(statistics); - maxmin = int32_stats->HasMinMax() && (!int32_stats->HasNullCount() || int32_stats->null_count() == 0) - ? DB::Range(int32_stats->min(), true, int32_stats->max(), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::INT64) - { - auto int64_stats = statistics ? std::dynamic_pointer_cast(statistics) : nullptr; - maxmin = int64_stats && int64_stats->HasMinMax() && (!int64_stats->HasNullCount() || int64_stats->null_count() == 0) - ? DB::Range(int64_stats->min(), true, int64_stats->max(), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::BOOLEAN) - { - auto bool_stats = std::dynamic_pointer_cast(statistics); - maxmin = bool_stats->HasMinMax() && (!bool_stats->HasNullCount() || bool_stats->null_count() == 0) - ? DB::Range(bool_stats->min(), true, bool_stats->max(), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::BYTE_ARRAY) - { - auto byte_array_stats = std::dynamic_pointer_cast(statistics); - maxmin = byte_array_stats->HasMinMax() && (!byte_array_stats->HasNullCount() || byte_array_stats->null_count() == 0) - ? DB::Range( - parquet::ByteArrayToString(byte_array_stats->min()), true, parquet::ByteArrayToString(byte_array_stats->max()), true) - : DB::Range::createWholeUniverse(); - } - else if (parquet_data_type == parquet::Type::type::FIXED_LEN_BYTE_ARRAY) - { - if (which_data_type.isFixedString()) - { - auto fixed_byte_array_stats = std::dynamic_pointer_cast(statistics); - maxmin = fixed_byte_array_stats->HasMinMax() - && (!fixed_byte_array_stats->HasNullCount() || fixed_byte_array_stats->null_count() == 0) - ? DB::Range( - parquet::FixedLenByteArrayToString(fixed_byte_array_stats->min(), column_type_length), - true, - parquet::FixedLenByteArrayToString(fixed_byte_array_stats->max(), column_type_length), - true) - : DB::Range::createWholeUniverse(); - } - } - - return maxmin; -} - - } #endif diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h index 80df5a8fa351..5a746c4b0e53 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ParquetFormatFile.h @@ -52,21 +52,16 @@ class ParquetFormatFile : public FormatFile std::optional getTotalRows() override; bool supportSplit() const override { return true; } - DB::String getFileFormat() const override { return "parquet"; } + + DB::String getFileFormat() const override { return "Parquet"; } private: bool use_local_format; std::mutex mutex; std::optional total_rows; - bool enable_row_group_maxmin_index; + std::vector collectRequiredRowGroups(int & total_row_groups); std::vector collectRequiredRowGroups(DB::ReadBuffer * read_buffer, int & total_row_groups); - bool checkRowGroupIfRequired(parquet::RowGroupMetaData & meta); - DB::Range getColumnMaxMin( - std::shared_ptr statistics, - parquet::Type::type parquet_data_type, - DB::DataTypePtr data_type, - Int32 column_type_length); }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp index 1941a7bd17a9..8fe6bf92884a 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/ReadBufferBuilder.cpp @@ -380,35 +380,26 @@ class S3FileReadBufferBuilder : public ReadBufferBuilder std::string bucket = file_uri.getHost(); const auto client = getClient(bucket); std::string key = file_uri.getPath().substr(1); - size_t object_size = DB::S3::getObjectSize(*client, bucket, key, ""); + DB::S3::ObjectInfo object_info = DB::S3::getObjectInfo(*client, bucket, key, ""); + size_t object_size = object_info.size; + Int64 object_modified_time = object_info.last_modification_time; if (new_settings.enable_filesystem_cache) { - const auto & settings = context->getSettingsRef(); - String accept_cache_time_str = getSetting(settings, "", "spark.kylin.local-cache.accept-cache-time"); - Int64 accept_cache_time; - if (accept_cache_time_str.empty()) - { - accept_cache_time = DateTimeUtil::currentTimeMillis(); - } - else - { - accept_cache_time = std::stoll(accept_cache_time_str); - } auto file_cache_key = DB::FileCacheKey(key); auto last_cache_time = files_cache_time_map.get(file_cache_key); // quick check if (last_cache_time != std::nullopt && last_cache_time.has_value()) { - if (last_cache_time.value() < accept_cache_time) + if (last_cache_time.value() < object_modified_time*1000l) //second to milli second { - files_cache_time_map.update_cache_time(file_cache_key, key, accept_cache_time, file_cache); + files_cache_time_map.update_cache_time(file_cache_key, key, object_modified_time*1000l, file_cache); } } else { - files_cache_time_map.update_cache_time(file_cache_key, key, accept_cache_time, file_cache); + files_cache_time_map.update_cache_time(file_cache_key, key, object_modified_time*1000l, file_cache); } } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp index fe8007586627..3f947573f9de 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.cpp @@ -68,7 +68,7 @@ static DB::Block getRealHeader(const DB::Block & header) SubstraitFileSource::SubstraitFileSource( DB::ContextPtr context_, const DB::Block & header_, const substrait::ReadRel::LocalFiles & file_infos) - : DB::ISource(getRealHeader(header_), false), context(context_), output_header(header_) + : DB::SourceWithKeyCondition(getRealHeader(header_), false), context(context_), output_header(header_) { if (file_infos.items_size()) { @@ -123,13 +123,20 @@ SubstraitFileSource::SubstraitFileSource( } } -void SubstraitFileSource::applyFilters(std::vector filters_) const +void SubstraitFileSource::setKeyCondition(const DB::ActionsDAG::NodeRawConstPtrs & nodes, DB::ContextPtr context_) { - for (size_t i = 0; i < files.size(); ++i) - { - FormatFilePtr file = files[i]; - file->setFilters(filters_); - } + const auto & keys = to_read_header; + std::unordered_map node_name_to_input_column; + for (const auto & column : keys.getColumnsWithTypeAndName()) + node_name_to_input_column.insert({column.name, column}); + + auto filter_actions_dag = DB::ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_column, context); + key_condition = std::make_shared( + filter_actions_dag, + context_, + keys.getNames(), + std::make_shared(std::make_shared(keys.getColumnsWithTypeAndName())), + DB::NameSet{}); } std::vector SubstraitFileSource::getPartitionKeys() const @@ -139,7 +146,7 @@ std::vector SubstraitFileSource::getPartitionKeys() const DB::String SubstraitFileSource::getFileFormat() const { - return files.size() > 0 ? files[0]->getFileFormat() : "unknown"; + return files.size() > 0 ? files[0]->getFileFormat() : "Unknown"; } DB::Chunk SubstraitFileSource::generate() @@ -208,6 +215,8 @@ bool SubstraitFileSource::tryPrepareReader() } else file_reader = std::make_unique(current_file, context, to_read_header, flatten_output_header); + + file_reader->applyKeyCondition(key_condition); return true; } @@ -466,7 +475,6 @@ NormalFileReader::NormalFileReader( reader = std::make_unique(*pipeline); } - bool NormalFileReader::pull(DB::Chunk & chunk) { DB::Chunk tmp_chunk; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h index 03e61217e4fb..5d0a2d69e9a4 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSource.h @@ -38,6 +38,7 @@ class FileReaderWrapper explicit FileReaderWrapper(FormatFilePtr file_) : file(file_) { } virtual ~FileReaderWrapper() = default; virtual bool pull(DB::Chunk & chunk) = 0; + virtual void applyKeyCondition(std::shared_ptr /*key_condition*/) { } protected: FormatFilePtr file; @@ -52,8 +53,14 @@ class NormalFileReader : public FileReaderWrapper public: NormalFileReader(FormatFilePtr file_, DB::ContextPtr context_, const DB::Block & to_read_header_, const DB::Block & output_header_); ~NormalFileReader() override = default; + bool pull(DB::Chunk & chunk) override; + void applyKeyCondition(std::shared_ptr key_condition) override + { + input_format->input->setKeyCondition(key_condition); + } + private: DB::ContextPtr context; DB::Block to_read_header; @@ -87,7 +94,7 @@ class ConstColumnsFileReader : public FileReaderWrapper size_t block_size; }; -class SubstraitFileSource : public DB::ISource +class SubstraitFileSource : public DB::SourceWithKeyCondition { public: SubstraitFileSource(DB::ContextPtr context_, const DB::Block & header_, const substrait::ReadRel::LocalFiles & file_infos); @@ -95,7 +102,8 @@ class SubstraitFileSource : public DB::ISource String getName() const override { return "SubstraitFileSource"; } - void applyFilters(std::vector filters) const; + void setKeyCondition(const DB::ActionsDAG::NodeRawConstPtrs & nodes, DB::ContextPtr context_) override; + std::vector getPartitionKeys() const; DB::String getFileFormat() const; @@ -115,7 +123,6 @@ class SubstraitFileSource : public DB::ISource /// E.g. if parquet file schema is `info struct`, and output_header is `info Tuple(name String, age Int32)` /// then there is not need to flatten `info` column, because null value of `info` column will be represented as null value of `info.name` and `info.age`, which is obviously wrong. std::unordered_set columns_to_skip_flatten; - std::vector filters; UInt32 current_file_index = 0; std::unique_ptr file_reader; diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.cpp b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.cpp index 3172a1ec8657..0a731ef756da 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.cpp +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.cpp @@ -44,7 +44,7 @@ namespace local_engine { SubstraitFileSourceStep::SubstraitFileSourceStep(DB::ContextPtr context_, DB::Pipe pipe_, const String &) - : SourceStepWithFilter(DB::DataStream{.header = pipe_.getHeader()}), pipe(std::move(pipe_)), context(context_) + : SourceStepWithFilter(DB::DataStream{.header = pipe_.getHeader()}), pipe(std::move(pipe_)), context(context_) { DB::Processors processors = pipe.getProcessors(); for (size_t i = 0; i < processors.size(); ++i) @@ -59,7 +59,6 @@ SubstraitFileSourceStep::SubstraitFileSourceStep(DB::ContextPtr context_, DB::Pi } } - void SubstraitFileSourceStep::initializePipeline(DB::QueryPipelineBuilder & pipeline, const DB::BuildQueryPipelineSettings &) { for (const auto & processor : pipe.getProcessors()) @@ -68,105 +67,12 @@ void SubstraitFileSourceStep::initializePipeline(DB::QueryPipelineBuilder & pipe pipeline.init(std::move(pipe)); } -DB::NamesAndTypesList SubstraitFileSourceStep::extractParquetFileColumnPathAndTypeForComplexType(const DB::String & column_name, const DB::DataTypePtr & column_type) -{ - DB::NamesAndTypesList path_and_type; - auto extract_path_and_type = [&](const DB::String & name, const DB::DataTypePtr & type, DB::NamesAndTypesList & result) -> void - { - DB::WhichDataType type_which(type); - if (type_which.isMap() || type_which.isArray() || type_which.isTuple()) - { - DB::NamesAndTypesList names_and_types = extractParquetFileColumnPathAndTypeForComplexType(name, type); - result.insert(result.end(), names_and_types.begin(), names_and_types.end()); - } - else - { - DB::NameAndTypePair path_type_pair(name, type); - result.push_back(path_type_pair); - } - }; - - DB::WhichDataType which_type(column_type); - if (which_type.isMap()) - { - const DB::DataTypeMap * map_type = dynamic_cast(column_type.get()); - DB::String key_path = column_name + ".key_value.key"; - DB::String value_path = column_name + ".key_value.value"; - DB::NameAndTypePair key_path_type_pair(key_path, map_type->getKeyType()); - path_and_type.push_back(key_path_type_pair); - extract_path_and_type(value_path, removeNullable(map_type->getValueType()), path_and_type); - } - else if (which_type.isArray()) - { - const DB::DataTypeArray * array_type = dynamic_cast(column_type.get()); - DB::String element_path = column_name + ".list.element"; - DB::DataTypePtr element_type = array_type->getNestedType(); - extract_path_and_type(element_path, removeNullable(element_type), path_and_type); - } - else if (which_type.isTuple()) - { - const DB::DataTypeTuple * tuple_type = dynamic_cast(column_type.get()); - DB::Names names = tuple_type->getElementNames(); - for (size_t i = 0; i < names.size(); ++i) - extract_path_and_type(column_name + "." + names[i], removeNullable(tuple_type->getElement(i)), path_and_type); - } - else - throw DB::Exception(DB::ErrorCodes::TYPE_MISMATCH, "Column {} type is not map/array/tuple, which not supported", column_name); - - return path_and_type; - -} - void SubstraitFileSourceStep::applyFilters() { - if (filter_dags.size() == 0 || filter_nodes.nodes.size() == 0) - return; - std::vector filters; - const DB::Block header = pipe.getHeader(); - const DB::ColumnsWithTypeAndName columns = header.getColumnsWithTypeAndName(); - std::unordered_map node_name_to_input_column; - DB::NamesAndTypesList filter_column_keys; - for (size_t i=0; i < columns.size(); ++i) - { - DB::ColumnWithTypeAndName column = columns[i]; - auto ret = std::find(partition_keys.begin(), partition_keys.end(), column.name); - if (ret == partition_keys.end()) - { - node_name_to_input_column.insert({column.name, column}); - DB::DataTypePtr column_type_non_nullable = removeNullable(column.type); - DB::WhichDataType which_type(column_type_non_nullable); - if (which_type.isMap() || which_type.isArray() || which_type.isTuple()) - { - if (file_format == "parquet") - { - DB::NamesAndTypesList names_and_types = extractParquetFileColumnPathAndTypeForComplexType(column.name, column_type_non_nullable); - filter_column_keys.insert(filter_column_keys.end(), names_and_types.begin(), names_and_types.end()); - } - } - else - { - DB::NameAndTypePair name_and_type(column.name, column.type); - filter_column_keys.push_back(name_and_type); - } - } - } - - if (!filter_column_keys.empty()) - { - std::shared_ptr filter_expr = std::make_shared(filter_dags[0], DB::ExpressionActionsSettings::fromContext(context)); - DB::ActionsDAGPtr filter_actions_dag = DB::ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, node_name_to_input_column, context); - DB::KeyCondition filter_condition(filter_actions_dag, context, filter_column_keys.getNames(), filter_expr, DB::NameSet{}); - SourceFilter filter{filter_condition, filter_column_keys}; - filters.push_back(filter); - } - DB::Processors processors = pipe.getProcessors(); - for (size_t i = 0; i < processors.size(); ++i) + for (const auto & processor : pipe.getProcessors()) { - DB::ProcessorPtr processor = processors[i]; - const SubstraitFileSource * source = static_cast(processor.get()); - if (source) - source->applyFilters(filters); + if (auto * source = dynamic_cast(processor.get())) + source->setKeyCondition(filter_nodes.nodes, context); } } - } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.h b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.h index 9d15fb72482c..b74599fe53a0 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/SubstraitFileSourceStep.h @@ -41,14 +41,6 @@ class SubstraitFileSourceStep : public DB::SourceStepWithFilter DB::ContextPtr context; std::vector partition_keys; DB::String file_format; - - DB::NamesAndTypesList extractParquetFileColumnPathAndTypeForComplexType(const DB::String & column_name, const DB::DataTypePtr & column_type); -}; - -struct SourceFilter -{ - DB::KeyCondition filter; - DB::NamesAndTypesList keys; }; } diff --git a/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.h b/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.h index 22304d77f2d6..ecff12b3b07e 100644 --- a/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.h +++ b/cpp-ch/local-engine/Storages/SubstraitSource/TextFormatFile.h @@ -32,7 +32,7 @@ class TextFormatFile : public FormatFile FormatFile::InputFormatPtr createInputFormat(const DB::Block & header) override; bool supportSplit() const override { return true; } - DB::String getFileFormat() const override { return "text"; } + DB::String getFileFormat() const override { return "HiveText"; } }; } diff --git a/cpp-ch/local-engine/local_engine_jni.cpp b/cpp-ch/local-engine/local_engine_jni.cpp index 5d6add257854..6813a5c9335f 100644 --- a/cpp-ch/local-engine/local_engine_jni.cpp +++ b/cpp-ch/local-engine/local_engine_jni.cpp @@ -653,7 +653,8 @@ JNIEXPORT jlong Java_io_glutenproject_vectorized_CHShuffleSplitterJniWrapper_nat jstring local_dirs, jint num_sub_dirs, jboolean prefer_spill, - jlong spill_threshold) + jlong spill_threshold, + jstring hash_algorithm) { LOCAL_ENGINE_JNI_METHOD_START std::string hash_exprs; @@ -694,7 +695,8 @@ JNIEXPORT jlong Java_io_glutenproject_vectorized_CHShuffleSplitterJniWrapper_nat .hash_exprs = hash_exprs, .out_exprs = out_exprs, .compress_method = jstring2string(env, codec), - .spill_threshold = static_cast(spill_threshold)}; + .spill_threshold = static_cast(spill_threshold), + .hash_algorithm = jstring2string(env, hash_algorithm)}; auto name = jstring2string(env, short_name); local_engine::SplitterHolder * splitter; if (prefer_spill) @@ -721,6 +723,7 @@ JNIEXPORT jlong Java_io_glutenproject_vectorized_CHShuffleSplitterJniWrapper_nat jint split_size, jstring codec, jlong spill_threshold, + jstring hash_algorithm, jobject pusher) { LOCAL_ENGINE_JNI_METHOD_START @@ -753,7 +756,8 @@ JNIEXPORT jlong Java_io_glutenproject_vectorized_CHShuffleSplitterJniWrapper_nat .hash_exprs = hash_exprs, .out_exprs = out_exprs, .compress_method = jstring2string(env, codec), - .spill_threshold = static_cast(spill_threshold)}; + .spill_threshold = static_cast(spill_threshold), + .hash_algorithm = jstring2string(env, hash_algorithm)}; auto name = jstring2string(env, short_name); local_engine::SplitterHolder * splitter; splitter = new local_engine::SplitterHolder{.splitter = std::make_unique(name, options, pusher)}; @@ -1060,12 +1064,14 @@ Java_io_glutenproject_vectorized_StorageJoinBuilder_nativeCleanBuildHashTable(JN // BlockSplitIterator JNIEXPORT jlong Java_io_glutenproject_vectorized_BlockSplitIterator_nativeCreate( - JNIEnv * env, jobject, jobject in, jstring name, jstring expr, jstring schema, jint partition_num, jint buffer_size) + JNIEnv * env, jobject, jobject in, jstring name, jstring expr, jstring schema, jint partition_num, jint buffer_size, jstring hash_algorithm) { LOCAL_ENGINE_JNI_METHOD_START local_engine::NativeSplitter::Options options; options.partition_nums = partition_num; options.buffer_size = buffer_size; + auto hash_algorithm_str = jstring2string(env, hash_algorithm); + options.hash_algorithm.swap(hash_algorithm_str); auto expr_str = jstring2string(env, expr); std::string schema_str; if (schema) diff --git a/cpp-ch/local-engine/proto/CMakeLists.txt b/cpp-ch/local-engine/proto/CMakeLists.txt index 055436b6fc32..f7bc1db2ad50 100644 --- a/cpp-ch/local-engine/proto/CMakeLists.txt +++ b/cpp-ch/local-engine/proto/CMakeLists.txt @@ -24,18 +24,27 @@ FOREACH(FIL ${protobuf_files}) LIST(APPEND SUBSTRAIT_HEADERS "${CMAKE_CURRENT_BINARY_DIR}/${FILE_NAME}.pb.h") ENDFOREACH() -add_custom_target( - generate_substrait - COMMAND ${CMAKE_CURRENT_BINARY_DIR}/../../../contrib/google-protobuf-cmake/protoc -I${CMAKE_CURRENT_SOURCE_DIR} -I${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src --cpp_out=${CMAKE_CURRENT_BINARY_DIR}/ ${protobuf_files} - DEPENDS protoc +# Generate Substrait headers +add_custom_command( + OUTPUT ${SUBSTRAIT_SRCS} ${SUBSTRAIT_HEADERS} + COMMAND + $ --cpp_out ${CMAKE_CURRENT_BINARY_DIR} + --proto_path ${CMAKE_CURRENT_SOURCE_DIR} + --proto_path ${ClickHouse_SOURCE_DIR}/contrib/google-protobuf/src + ${protobuf_files} + DEPENDS ${protobuf_files} COMMENT "Running cpp protocol buffer compiler" - VERBATIM ) - + VERBATIM) +add_custom_target(generate_substrait ALL DEPENDS ${SUBSTRAIT_SRCS} ${SUBSTRAIT_HEADERS}) set_source_files_properties(${SUBSTRAIT_SRCS} PROPERTIES GENERATED TRUE) add_library(substrait ${SUBSTRAIT_SRCS}) add_dependencies(substrait generate_substrait) -target_compile_options(substrait PUBLIC -fPIC -Wno-reserved-identifier -Wno-deprecated -Wno-shadow-field) +target_compile_options(substrait PUBLIC -fPIC + -Wno-reserved-identifier + -Wno-deprecated + -Wno-extra-semi-stmt + -Wno-used-but-marked-unused) target_include_directories(substrait SYSTEM BEFORE PRIVATE ${CMAKE_CURRENT_BINARY_DIR}) target_link_libraries(substrait ch_contrib::protobuf) diff --git a/cpp/CMake/BuildGTest.cmake b/cpp/CMake/BuildGTest.cmake index 96b4657ae986..7318b7bd7081 100644 --- a/cpp/CMake/BuildGTest.cmake +++ b/cpp/CMake/BuildGTest.cmake @@ -8,9 +8,7 @@ set(GLUTEN_GTEST_SOURCE_URL "https://github.com/google/googletest/archive/refs/tags/v${GLUTEN_GTEST_VERSION}.tar.gz" ) -if(DEFINED ENV{GLUTEN_GTEST_SOURCE_URL}) - set(GLUTEN_GTEST_SOURCE_URL "$ENV{GLUTEN_GTEST_SOURCE_URL}") -endif() +resolve_dependency_url(GTEST) message(STATUS "Building gtest from source") FetchContent_Declare( diff --git a/cpp/CMake/BuildGflags.cmake b/cpp/CMake/BuildGflags.cmake new file mode 100644 index 000000000000..4cb201115835 --- /dev/null +++ b/cpp/CMake/BuildGflags.cmake @@ -0,0 +1,43 @@ +# 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_guard(GLOBAL) + +set(GLUTEN_GFLAGS_BUILD_SHA256_CHECKSUM + 34af2f15cf7367513b352bdcd2493ab14ce43692d2dcd9dfc499492966c64dcf) +string(CONCAT GLUTEN_GFLAGS_SOURCE_URL + "https://github.com/gflags/gflags/archive/refs/tags/" + "v${GLUTEN_GFLAGS_VERSION}.tar.gz") + +resolve_dependency_url(GFLAGS) + +message(STATUS "Building gflags from source") +FetchContent_Declare( + gflags + URL ${GLUTEN_GFLAGS_SOURCE_URL} + URL_HASH SHA256=${GLUTEN_GFLAGS_BUILD_SHA256_CHECKSUM}) + +set(GFLAGS_BUILD_STATIC_LIBS ON) +set(GFLAGS_BUILD_SHARED_LIBS ON) +set(GFLAGS_BUILD_gflags_LIB ON) +# glog relies on the old `google` namespace +set(GFLAGS_NAMESPACE "google;gflags") + +FetchContent_MakeAvailable(gflags) + +# the flag has to be added to each target we build so adjust to settings choosen +# above +target_compile_options(gflags_static PRIVATE -Wno-cast-function-type) diff --git a/cpp/CMake/BuildGlog.cmake b/cpp/CMake/BuildGlog.cmake new file mode 100644 index 000000000000..cf405225c313 --- /dev/null +++ b/cpp/CMake/BuildGlog.cmake @@ -0,0 +1,53 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +include_guard(GLOBAL) + +set(GLUTEN_GLOG_BUILD_SHA256_CHECKSUM + 8a83bf982f37bb70825df71a9709fa90ea9f4447fb3c099e1d720a439d88bad6) +set(GLUTEN_GLOG_SOURCE_URL + "https://github.com/google/glog/archive/refs/tags/v${GLUTEN_GLOG_VERSION}.tar.gz" + ) + +resolve_dependency_url(GLOG) + +message(STATUS "Building glog from source") +FetchContent_Declare( + glog + URL ${GLUTEN_GLOG_SOURCE_URL} + URL_HASH SHA256=${GLUTEN_GLOG_BUILD_SHA256_CHECKSUM} + PATCH_COMMAND git apply ${CMAKE_CURRENT_LIST_DIR}/glog/glog-no-export.patch + && git apply ${CMAKE_CURRENT_LIST_DIR}/glog/glog-config.patch) + +set(BUILD_SHARED_LIBS OFF) +set(WITH_UNWIND OFF) +set(BUILD_TESTING OFF) +FetchContent_MakeAvailable(glog) +unset(BUILD_TESTING) +unset(BUILD_SHARED_LIBS) + +list(PREPEND CMAKE_MODULE_PATH ${CMAKE_CURRENT_LIST_DIR}/glog) +set(GLOG_INCLUDE_DIR ${glog_BINARY_DIR}) +string(TOUPPER "${CMAKE_BUILD_TYPE}" UPPERCASE_BUILD_TYPE) +if(UPPERCASE_BUILD_TYPE MATCHES "DEBUG") + set(GLOG_LIBRARY "${glog_BINARY_DIR}/libglogd.a") +else() + set(GLOG_LIBRARY "${glog_BINARY_DIR}/libglog.a") +endif() + +# These headers are missing from the include dir but adding the src dir causes +# issues with folly so we just copy it to the include dir +file(COPY ${glog_SOURCE_DIR}/src/glog/platform.h + DESTINATION ${glog_BINARY_DIR}/glog) +file(COPY ${glog_SOURCE_DIR}/src/glog/log_severity.h + DESTINATION ${glog_BINARY_DIR}/glog) diff --git a/cpp/CMake/BuildGoogleBenchmark.cmake b/cpp/CMake/BuildGoogleBenchmark.cmake index 3a5af880a00a..8efbb58eab74 100644 --- a/cpp/CMake/BuildGoogleBenchmark.cmake +++ b/cpp/CMake/BuildGoogleBenchmark.cmake @@ -24,9 +24,7 @@ set(GLUTEN_GBENCHMARK_SOURCE_URL "https://github.com/ursa-labs/thirdparty/releases/download/latest/gbenchmark-${GLUTEN_GBENCHMARK_BUILD_VERSION}.tar.gz") set(GLUTEN_GBENCHMARK_BUILD_SHA256_CHECKSUM "1f71c72ce08d2c1310011ea6436b31e39ccab8c2db94186d26657d41747c85d6") -if(DEFINED ENV{GLUTEN_GBENCHMARK_SOURCE_URL}) - set(GLUTEN_GBENCHMARK_SOURCE_URL "$ENV{GLUTEN_GBENCHMARK_SOURCE_URL}") -endif() +resolve_dependency_url(GBENCHMARK) set(GBENCHMARK_CMAKE_ARGS "-fPIC -w") @@ -34,7 +32,7 @@ message(STATUS "Building google benchmark from source") FetchContent_Declare( gbenchmark URL ${GLUTEN_GBENCHMARK_SOURCE_URL} - URL_HASH "SHA256=${GLUTEN_GBENCHMARK_BUILD_SHA256_CHECKSUM}" + URL_HASH "${GLUTEN_GBENCHMARK_BUILD_SHA256_CHECKSUM}" ) if (NOT gbenchmark_POPULATED) diff --git a/cpp/CMake/ConfigArrow.cmake b/cpp/CMake/ConfigArrow.cmake index db67115f3f90..d9633b27cc81 100644 --- a/cpp/CMake/ConfigArrow.cmake +++ b/cpp/CMake/ConfigArrow.cmake @@ -16,11 +16,11 @@ # under the License. if (${CMAKE_SYSTEM_NAME} STREQUAL "Darwin") - set(ARROW_SHARED_LIBRARY_SUFFIX ".1200.dylib") - set(ARROW_SHARED_LIBRARY_PARENT_SUFFIX ".1200.0.0.dylib") + set(ARROW_SHARED_LIBRARY_SUFFIX ".1400.dylib") + set(ARROW_SHARED_LIBRARY_PARENT_SUFFIX ".1400.1.0.dylib") else() - set(ARROW_SHARED_LIBRARY_SUFFIX ".so.1200") - set(ARROW_SHARED_LIBRARY_PARENT_SUFFIX ".so.1200.0.0") + set(ARROW_SHARED_LIBRARY_SUFFIX ".so.1400") + set(ARROW_SHARED_LIBRARY_PARENT_SUFFIX ".so.1400.1.0") endif() set(ARROW_LIB_NAME "arrow") @@ -32,46 +32,24 @@ function(FIND_ARROW_LIB LIB_NAME) if(NOT TARGET Arrow::${LIB_NAME}) set(ARROW_LIB_FULL_NAME ${CMAKE_SHARED_LIBRARY_PREFIX}${LIB_NAME}${ARROW_SHARED_LIBRARY_SUFFIX}) add_library(Arrow::${LIB_NAME} SHARED IMPORTED) - set_target_properties(Arrow::${LIB_NAME} - PROPERTIES IMPORTED_LOCATION "${root_directory}/releases/${ARROW_LIB_FULL_NAME}" - INTERFACE_INCLUDE_DIRECTORIES - "${root_directory}/releases/include") find_library(ARROW_LIB_${LIB_NAME} NAMES ${ARROW_LIB_FULL_NAME} PATHS ${ARROW_LIB_DIR} ${ARROW_LIB64_DIR} NO_DEFAULT_PATH) if(NOT ARROW_LIB_${LIB_NAME}) - message(FATAL_ERROR "Arrow Library Not Found: ${ARROW_LIB_FULL_NAME}") + message(FATAL_ERROR "Arrow library Not Found: ${ARROW_LIB_FULL_NAME}") else() - message(STATUS "Found Arrow Library: ${ARROW_LIB_${LIB_NAME}}") + message(STATUS "Found Arrow library: ${ARROW_LIB_${LIB_NAME}}") + set_target_properties(Arrow::${LIB_NAME} + PROPERTIES IMPORTED_LOCATION "${ARROW_LIB_${LIB_NAME}}" + INTERFACE_INCLUDE_DIRECTORIES + "${ARROW_HOME}/install/include") endif() file(COPY ${ARROW_LIB_${LIB_NAME}} DESTINATION ${root_directory}/releases/ FOLLOW_SYMLINK_CHAIN) endif() endfunction() -message(STATUS "Use existing ARROW libraries") - -set(ARROW_INSTALL_DIR "${ARROW_HOME}/arrow_install") +set(ARROW_INSTALL_DIR "${ARROW_HOME}/install") set(ARROW_LIB_DIR "${ARROW_INSTALL_DIR}/lib") set(ARROW_LIB64_DIR "${ARROW_INSTALL_DIR}/lib64") set(ARROW_INCLUDE_DIR "${ARROW_INSTALL_DIR}/include") - -message(STATUS "Set Arrow Library Directory in ${ARROW_LIB_DIR} or ${ARROW_LIB64_DIR}") -message(STATUS "Set Arrow Include Directory in ${ARROW_INCLUDE_DIR}") - -if(EXISTS ${ARROW_INCLUDE_DIR}/arrow) - set(ARROW_INCLUDE_SRC_DIR ${ARROW_INCLUDE_DIR}) -else() - message(FATAL_ERROR "Arrow headers not found in ${ARROW_INCLUDE_DIR}/arrow.") -endif() - -# Copy arrow headers -set(ARROW_INCLUDE_DST_DIR ${root_directory}/releases/include) - -string(TOUPPER "${BUILD_BENCHMARKS}" LOWERCASE_BUILD_BENCHMARKS) -set(ARROW_INCLUDE_SUB_DIR arrow parquet) -message(STATUS "Copy Arrow headers from ${ARROW_INCLUDE_SRC_DIR} to ${ARROW_INCLUDE_DST_DIR}") -file(MAKE_DIRECTORY ${ARROW_INCLUDE_DST_DIR}) -foreach(SUB_DIR ${ARROW_INCLUDE_SUB_DIR}) - file(COPY ${ARROW_INCLUDE_SRC_DIR}/${SUB_DIR} DESTINATION ${ARROW_INCLUDE_DST_DIR}) -endforeach() diff --git a/cpp/CMake/Findglog.cmake b/cpp/CMake/Findglog.cmake index 752647cb3357..b165fd80f3d7 100644 --- a/cpp/CMake/Findglog.cmake +++ b/cpp/CMake/Findglog.cmake @@ -1,37 +1,81 @@ -# Copyright (c) Facebook, Inc. and its affiliates. -# - Try to find Glog -# Once done, this will define +# 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 # -# GLOG_FOUND - system has Glog -# GLOG_INCLUDE_DIRS - the Glog include directories -# GLOG_LIBRARIES - link these to use Glog +# 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(GLUTEN_GLOG_MINIMUM_VERSION 0.4.0) +set(GLUTEN_GLOG_VERSION 0.6.0) + +if(NOT BUILD_GLOG) + include(FindPackageHandleStandardArgs) + include(SelectLibraryConfigurations) + + find_library(GLOG_LIBRARY_RELEASE glog + PATHS ${GLOG_LIBRARYDIR}) + find_library(GLOG_LIBRARY_DEBUG glogd + PATHS ${GLOG_LIBRARYDIR}) + + find_path(GLOG_INCLUDE_DIR glog/logging.h + PATHS ${GLOG_INCLUDEDIR}) -include(FindPackageHandleStandardArgs) -include(SelectLibraryConfigurations) + select_library_configurations(GLOG) -find_library(GLOG_LIBRARY_RELEASE glog - PATHS ${GLOG_LIBRARYDIR}) -find_library(GLOG_LIBRARY_DEBUG glogd - PATHS ${GLOG_LIBRARYDIR}) + find_package_handle_standard_args(glog DEFAULT_MSG + GLOG_LIBRARY + GLOG_INCLUDE_DIR) -find_path(GLOG_INCLUDE_DIR glog/logging.h - PATHS ${GLOG_INCLUDEDIR}) + mark_as_advanced( + GLOG_LIBRARY + GLOG_INCLUDE_DIR) +endif() -select_library_configurations(GLOG) +if(NOT glog_FOUND) + include(BuildGlog) +endif() -find_package_handle_standard_args(glog DEFAULT_MSG - GLOG_LIBRARY - GLOG_INCLUDE_DIR) +get_filename_component(libglog_ext ${GLOG_LIBRARY} EXT) +if(libglog_ext STREQUAL ".a") + set(libglog_type STATIC) + set(libgflags_component static) +else() + set(libglog_type SHARED) + set(libgflags_component shared) +endif() -mark_as_advanced( - GLOG_LIBRARY - GLOG_INCLUDE_DIR) +# glog::glog may already exist. Use google::glog to avoid conflicts. +add_library(google::glog ${libglog_type} IMPORTED) +set_target_properties(google::glog PROPERTIES INTERFACE_INCLUDE_DIRECTORIES "${GLOG_INCLUDE_DIR}") +set_target_properties(google::glog PROPERTIES IMPORTED_LINK_INTERFACE_LANGUAGES "C" IMPORTED_LOCATION "${GLOG_LIBRARY}") -set(GLOG_LIBRARIES ${GLOG_LIBRARY}) -set(GLOG_INCLUDE_DIRS ${GLOG_INCLUDE_DIR}) +set(GLUTEN_GFLAGS_VERSION 2.2.2) +find_package(gflags ${GLUTEN_GFLAGS_VERSION} CONFIG COMPONENTS ${libgflags_component}) + +if(NOT gflags_FOUND AND glog_FOUND) + message(FATAL_ERROR "Glog found but Gflags not found. Set BUILD_GLOG=ON and reload cmake.") +endif() -if (NOT TARGET glog::glog) - add_library(glog::glog UNKNOWN IMPORTED) - set_target_properties(glog::glog PROPERTIES INTERFACE_INCLUDE_DIRECTORIES "${GLOG_INCLUDE_DIRS}") - set_target_properties(glog::glog PROPERTIES IMPORTED_LINK_INTERFACE_LANGUAGES "C" IMPORTED_LOCATION "${GLOG_LIBRARIES}") +if(gflags_FOUND) + if(NOT TARGET gflags::gflags_${libgflags_component} AND NOT TARGET gflags_${libgflags_component}) + message(FATAL_ERROR "Found Gflags but missing component gflags_${libgflags_component}") + endif() + if(TARGET gflags::gflags_${libgflags_component}) + set_target_properties(google::glog PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES gflags::gflags_${libgflags_component}) + else() + set_target_properties(google::glog PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES gflags_${libgflags_component}) + endif() +else() + include(BuildGflags) + set_target_properties(google::glog PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES gflags_static) endif() diff --git a/cpp/CMake/ResolveDependency.cmake b/cpp/CMake/ResolveDependency.cmake new file mode 100755 index 000000000000..2f44af607ccd --- /dev/null +++ b/cpp/CMake/ResolveDependency.cmake @@ -0,0 +1,44 @@ +# 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. + +function(set_with_default var_name envvar_name default) + if(DEFINED ENV{${envvar_name}}) + set(${var_name} + $ENV{${envvar_name}} + PARENT_SCOPE) + else() + set(${var_name} + ${default} + PARENT_SCOPE) + endif() +endfunction() + +macro(resolve_dependency_url dependency_name) + # Prepend prefix for default checksum. + string(PREPEND GLUTEN_${dependency_name}_BUILD_SHA256_CHECKSUM "SHA256=") + + set_with_default( + GLUTEN_${dependency_name}_SOURCE_URL GLUTEN_${dependency_name}_URL + ${GLUTEN_${dependency_name}_SOURCE_URL}) + if(DEFINED ENV{GLUTEN_${dependency_name}_URL}) + set_with_default(GLUTEN_${dependency_name}_BUILD_SHA256_CHECKSUM + GLUTEN_${dependency_name}_SHA256 "") + if(DEFINED ENV{GLUTEN_${dependency_name}_SHA256}) + string(PREPEND GLUTEN_${dependency_name}_BUILD_SHA256_CHECKSUM "SHA256=") + endif() + endif() +endmacro() diff --git a/cpp/CMake/Simdjson.cmake b/cpp/CMake/Simdjson.cmake new file mode 100755 index 000000000000..f78090e7eed2 --- /dev/null +++ b/cpp/CMake/Simdjson.cmake @@ -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_guard(GLOBAL) +include(FetchContent) + +set(GLUTEN_SIMDJSON_VERSION 3.1.5) +set(GLUTEN_SIMDJSON_BUILD_SHA256_CHECKSUM + 5b916be17343324426fc467a4041a30151e481700d60790acfd89716ecc37076) +set(GLUTEN_SIMDJSON_SOURCE_URL + "https://github.com/simdjson/simdjson/archive/refs/tags/v${GLUTEN_SIMDJSON_VERSION}.tar.gz" +) + +resolve_dependency_url(SIMDJSON) + +message(STATUS "Building simdjson from source") + +FetchContent_Declare( + simdjson + URL ${GLUTEN_SIMDJSON_SOURCE_URL} + URL_HASH ${GLUTEN_SIMDJSON_BUILD_SHA256_CHECKSUM}) + +FetchContent_MakeAvailable(simdjson) +target_compile_options(simdjson PUBLIC -fPIC) diff --git a/cpp/CMake/glog/glog-config.patch b/cpp/CMake/glog/glog-config.patch new file mode 100644 index 000000000000..4851461fc83b --- /dev/null +++ b/cpp/CMake/glog/glog-config.patch @@ -0,0 +1,20 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +--- a/glog-config.cmake.in ++++ b/glog-config.cmake.in +@@ -10,4 +10,3 @@ include (${CMAKE_CURRENT_LIST_DIR}/glog-modules.cmake) + @gflags_DEPENDENCY@ + @Unwind_DEPENDENCY@ + +-include (${CMAKE_CURRENT_LIST_DIR}/glog-targets.cmake) diff --git a/cpp/CMake/glog/glog-no-export.patch b/cpp/CMake/glog/glog-no-export.patch new file mode 100644 index 000000000000..1ae4ce7cbd19 --- /dev/null +++ b/cpp/CMake/glog/glog-no-export.patch @@ -0,0 +1,32 @@ +# Copyright (c) Facebook, Inc. and its affiliates. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -1025,7 +1025,7 @@ write_basic_package_version_file ( + ${CMAKE_CURRENT_BINARY_DIR}/glog-config-version.cmake + COMPATIBILITY SameMajorVersion) + +-export (TARGETS glog NAMESPACE glog:: FILE glog-targets.cmake) ++# export (TARGETS glog NAMESPACE glog:: FILE glog-targets.cmake) + export (PACKAGE glog) + + get_filename_component (_PREFIX "${CMAKE_INSTALL_PREFIX}" ABSOLUTE) +@@ -1076,5 +1076,5 @@ install (DIRECTORY ${_glog_BINARY_CMake_DATADIR} + FILES_MATCHING PATTERN "*.cmake" + ) + +-install (EXPORT glog-targets NAMESPACE glog:: DESTINATION +- ${_glog_CMake_INSTALLDIR}) ++# install (EXPORT glog-targets NAMESPACE glog:: DESTINATION ++# ${_glog_CMake_INSTALLDIR}) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 7f588726a883..09f7a42b893d 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -43,20 +43,32 @@ option(BUILD_EXAMPLES "Build Examples" OFF) option(BUILD_BENCHMARKS "Build Benchmarks" OFF) option(BUILD_PROTOBUF "Build Protobuf from Source" OFF) option(BUILD_JEMALLOC "Build Jemalloc from Source" 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) +option(ENABLE_GCS "Enable GCS" OFF) option(ENABLE_S3 "Enable S3" OFF) option(ENABLE_HDFS "Enable HDFS" OFF) +option(ENABLE_ORC "Enable ORC" OFF) set(root_directory ${PROJECT_BINARY_DIR}) get_filename_component(GLUTEN_HOME ${CMAKE_SOURCE_DIR} DIRECTORY) -if (NOT DEFINED ARROW_HOME) - set(ARROW_HOME ${GLUTEN_HOME}/ep/build-arrow/build) +if (NOT DEFINED VELOX_HOME) + set(VELOX_HOME ${GLUTEN_HOME}/ep/build-velox/build/velox_ep) + message(STATUS "Set VELOX_HOME to ${VELOX_HOME}") endif() +if (${CMAKE_BUILD_TYPE} STREQUAL "Debug") + set(ARROW_HOME ${VELOX_HOME}/_build/debug/third_party/arrow_ep) +else() + set(ARROW_HOME ${VELOX_HOME}/_build/release/third_party/arrow_ep) +endif() + +include(ResolveDependency) + # # Compiler flags # @@ -86,6 +98,9 @@ add_compile_options(-Wno-error=unused-variable) if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "GNU") add_compile_options(-Wno-error=unused-but-set-variable) + if(CMAKE_CXX_COMPILER_VERSION VERSION_LESS 11) + add_compile_options(-Wno-error=maybe-uninitialized) + endif() elseif ("${CMAKE_CXX_COMPILER_ID}" STREQUAL "Clang") # Experimental add_compile_options(-Wno-implicit-int-float-conversion) @@ -100,6 +115,7 @@ if (${CMAKE_SYSTEM_NAME} MATCHES "Darwin") add_compile_options(-Wno-macro-redefined) add_compile_options(-Wno-nullability-completeness) add_compile_options(-Wno-pessimizing-move) + add_compile_options(-Wno-mismatched-tags) endif() # @@ -113,6 +129,8 @@ find_package(Threads REQUIRED) find_package(JNI REQUIRED) +find_package(glog REQUIRED) + if(BUILD_TESTS) set(GLUTEN_GTEST_MIN_VERSION "1.13.0") find_package(GTest ${GLUTEN_GTEST_MIN_VERSION} CONFIG) @@ -121,7 +139,6 @@ if(BUILD_TESTS) endif() include(GoogleTest) enable_testing() - find_package(glog REQUIRED) endif() function(ADD_TEST_CASE TEST_NAME) @@ -180,6 +197,10 @@ if(ENABLE_IAA) add_definitions(-DGLUTEN_ENABLE_IAA) endif() +if(ENABLE_ORC) + add_definitions(-DGLUTEN_ENABLE_ORC) +endif() + # # Subdirectories # diff --git a/cpp/compile.sh b/cpp/compile.sh index 2810c29020a1..986d0ec8cd29 100755 --- a/cpp/compile.sh +++ b/cpp/compile.sh @@ -25,18 +25,14 @@ BUILD_JEMALLOC=OFF BUILD_PROTOBUF=OFF ENABLE_QAT=OFF ENABLE_HBM=OFF +ENABLE_GCS=OFF ENABLE_S3=OFF ENABLE_HDFS=OFF -ARROW_HOME= VELOX_HOME= NPROC=$(nproc --ignore=2) for arg in "$@"; do case $arg in - --arrow_home=*) - ARROW_HOME=("${arg#*=}") - shift # Remove argument name from processing - ;; --velox_home=*) VELOX_HOME=("${arg#*=}") shift # Remove argument name from processing @@ -77,6 +73,10 @@ for arg in "$@"; do BUILD_PROTOBUF=("${arg#*=}") shift # Remove argument name from processing ;; + --enable_gcs=*) + ENABLE_GCS=("${arg#*=}") + shift # Remove argument name from processing + ;; --enable_s3=*) ENABLE_S3=("${arg#*=}") shift # Remove argument name from processing @@ -97,11 +97,6 @@ CURRENT_DIR=$( pwd ) -#gluten cpp will find arrow/parquet lib from ARROW_HOME -if [ "$ARROW_HOME" == "" ]; then - ARROW_HOME="$CURRENT_DIR/../ep/build-arrow/build" -fi - #gluten cpp will find velox lib from VELOX_HOME if [ "$VELOX_HOME" == "" ]; then VELOX_HOME="$CURRENT_DIR/../ep/build-velox/build/velox_ep" @@ -109,7 +104,6 @@ fi echo "Building gluten cpp part..." echo "CMAKE Arguments:" -echo "ARROW_HOME=${ARROW_HOME}" echo "VELOX_HOME=${VELOX_HOME}" echo "BUILD_TYPE=${BUILD_TYPE}" echo "BUILD_VELOX_BACKEND=${BUILD_VELOX_BACKEND}" @@ -119,6 +113,7 @@ echo "BUILD_BENCHMARKS=${BUILD_BENCHMARKS}" echo "BUILD_JEMALLOC=${BUILD_JEMALLOC}" echo "ENABLE_HBM=${ENABLE_HBM}" echo "BUILD_PROTOBUF=${BUILD_PROTOBUF}" +echo "ENABLE_GCS=${ENABLE_GCS}" echo "ENABLE_S3=${ENABLE_S3}" echo "ENABLE_HDFS=${ENABLE_HDFS}" @@ -130,7 +125,6 @@ cd build cmake .. \ -DBUILD_TESTS=${BUILD_TESTS} \ -DBUILD_EXAMPLES=${BUILD_EXAMPLES} \ - -DARROW_HOME=${ARROW_HOME} \ -DBUILD_JEMALLOC=${BUILD_JEMALLOC} \ -DBUILD_VELOX_BACKEND=${BUILD_VELOX_BACKEND} \ -DVELOX_HOME=${VELOX_HOME} \ @@ -139,6 +133,7 @@ cmake .. \ -DBUILD_PROTOBUF=${BUILD_PROTOBUF} \ -DENABLE_QAT=${ENABLE_QAT} \ -DENABLE_HBM=${ENABLE_HBM} \ + -DENABLE_GCS=${ENABLE_GCS} \ -DENABLE_S3=${ENABLE_S3} \ -DENABLE_HDFS=${ENABLE_HDFS} make -j$NPROC diff --git a/cpp/core/CMakeLists.txt b/cpp/core/CMakeLists.txt index df04490051d4..fb7c714a3ef8 100644 --- a/cpp/core/CMakeLists.txt +++ b/cpp/core/CMakeLists.txt @@ -184,7 +184,7 @@ set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-deprecated-declarations -Wno-attrib set(SPARK_COLUMNAR_PLUGIN_SRCS ${SUBSTRAIT_PROTO_SRCS} ${GLUTEN_PROTO_SRCS} - compute/ExecutionCtx.cc + compute/Runtime.cc compute/ProtobufUtils.cc compute/ResultIterator.cc config/GlutenConfig.cc @@ -194,13 +194,15 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS memory/ArrowMemoryPool.cc memory/ColumnarBatch.cc operators/writer/ArrowWriter.cc + shuffle/Options.cc shuffle/ShuffleReader.cc shuffle/ShuffleWriter.cc shuffle/Partitioner.cc shuffle/FallbackRangePartitioner.cc shuffle/HashPartitioner.cc shuffle/RoundRobinPartitioner.cc - shuffle/SinglePartPartitioner.cc + shuffle/SinglePartitioner.cc + shuffle/Partitioning.cc shuffle/PartitionWriterCreator.cc shuffle/LocalPartitionWriter.cc shuffle/rss/RemotePartitionWriter.cc @@ -209,6 +211,7 @@ set(SPARK_COLUMNAR_PLUGIN_SRCS utils/Compression.cc utils/DebugOut.cc utils/StringUtil.cc + utils/ObjectStore.cc jni/JniError.cc jni/JniCommon.cc) diff --git a/cpp/core/benchmarks/CompressionBenchmark.cc b/cpp/core/benchmarks/CompressionBenchmark.cc index 590ae236d4d4..3e2396818235 100644 --- a/cpp/core/benchmarks/CompressionBenchmark.cc +++ b/cpp/core/benchmarks/CompressionBenchmark.cc @@ -34,7 +34,7 @@ #include #include "shuffle/ShuffleWriter.h" -#include "utils/compression.h" +#include "utils/Compression.h" #include "utils/macros.h" void printTrace(void) { @@ -65,62 +65,6 @@ const int32_t kQplGzip = 2; const int32_t kLZ4 = 3; const int32_t kZstd = 4; -class MyMemoryPool final : public arrow::MemoryPool { - public: - explicit MyMemoryPool() {} - - Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override { - RETURN_NOT_OK(pool_->Allocate(size, out)); - stats_.UpdateAllocatedBytes(size); - // std::cout << "Allocate: size = " << size << " addr = " << std::hex << - // (uint64_t)*out << std::dec << std::endl; print_trace(); - return arrow::Status::OK(); - } - - Status Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) override { - // auto old_ptr = *ptr; - RETURN_NOT_OK(pool_->Reallocate(oldSize, newSize, ptr)); - stats_.UpdateAllocatedBytes(newSize - oldSize); - // std::cout << "Reallocate: old_size = " << old_size << " old_ptr = " << - // std::hex << (uint64_t)old_ptr << std::dec << " new_size = " << new_size - // << " addr = " << std::hex << (uint64_t)*ptr << std::dec << std::endl; - // print_trace(); - return arrow::Status::OK(); - } - - void Free(uint8_t* buffer, int64_t size, int64_t alignment) override { - pool_->Free(buffer, size); - stats_.UpdateAllocatedBytes(-size); - // std::cout << "Free: size = " << size << " addr = " << std::hex << - // (uint64_t)buffer - // << std::dec << std::endl; print_trace(); - } - - int64_t bytes_allocated() const override { - return stats_.bytes_allocated(); - } - - int64_t max_memory() const override { - return pool_->max_memory(); - } - - std::string backend_name() const override { - return pool_->backend_name(); - } - - int64_t total_bytes_allocated() const override { - return pool_->total_bytes_allocated(); - } - - int64_t num_allocations() const override { - throw pool_->num_allocations(); - } - - private: - arrow::MemoryPool* pool_ = arrow::default_memory_pool(); - arrow::internal::MemoryPoolStats stats_; -}; - class BenchmarkCompression { public: explicit BenchmarkCompression(const std::string& fileName, uint32_t compressBufferSize) { @@ -195,8 +139,7 @@ class BenchmarkCompression { default: throw GlutenException("Codec not supported. Only support LZ4 or QATGzip"); } - std::shared_ptr pool = std::make_shared(); - ipcWriteOptions.memory_pool = pool.get(); + ipcWriteOptions.memory_pool = arrow::default_memory_pool(); int64_t elapseRead = 0; int64_t numBatches = 0; diff --git a/cpp/core/compute/ExecutionCtx.h b/cpp/core/compute/ExecutionCtx.h deleted file mode 100644 index d0fd37415c15..000000000000 --- a/cpp/core/compute/ExecutionCtx.h +++ /dev/null @@ -1,173 +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 "compute/ProtobufUtils.h" -#include "compute/ResultIterator.h" -#include "memory/ArrowMemoryPool.h" -#include "memory/ColumnarBatch.h" -#include "memory/MemoryManager.h" -#include "operators/c2r/ColumnarToRow.h" -#include "operators/r2c/RowToColumnar.h" -#include "operators/serializer/ColumnarBatchSerializer.h" -#include "operators/writer/Datasource.h" -#include "shuffle/ShuffleReader.h" -#include "shuffle/ShuffleWriter.h" -#include "substrait/plan.pb.h" -#include "utils/DebugOut.h" - -namespace gluten { - -using ResourceHandle = int64_t; -constexpr static ResourceHandle kInvalidResourceHandle = -1; - -class ResultIterator; - -struct SparkTaskInfo { - int32_t stageId{0}; - int32_t partitionId{0}; - // Same as TID. - int64_t taskId{0}; - - std::string toString() const { - return "[Stage: " + std::to_string(stageId) + " TID: " + std::to_string(taskId) + "]"; - } -}; - -/// ExecutionCtx is stateful and manager all kinds of native resources' lifecycle during execute a computation fragment. -class ExecutionCtx : public std::enable_shared_from_this { - public: - using Factory = std::function&)>; - static void registerFactory(const std::string& kind, Factory factory); - static ExecutionCtx* create( - const std::string& kind, - const std::unordered_map& sessionConf = {}); - static void release(ExecutionCtx*); - - ExecutionCtx() = default; - ExecutionCtx(const std::unordered_map& confMap) : confMap_(confMap) {} - virtual ~ExecutionCtx() = default; - - virtual ResourceHandle createResultIterator( - MemoryManager* memoryManager, - const std::string& spillDir, - const std::vector>& inputs, - const std::unordered_map& sessionConf) = 0; - virtual ResourceHandle addResultIterator(std::shared_ptr) = 0; - virtual std::shared_ptr getResultIterator(ResourceHandle) = 0; - virtual void releaseResultIterator(ResourceHandle) = 0; - - virtual ResourceHandle addBatch(std::shared_ptr) = 0; - virtual std::shared_ptr getBatch(ResourceHandle) = 0; - virtual ResourceHandle createOrGetEmptySchemaBatch(int32_t numRows) = 0; - virtual void releaseBatch(ResourceHandle) = 0; - virtual ResourceHandle select(MemoryManager*, ResourceHandle, std::vector) = 0; - - void parsePlan(const uint8_t* data, int32_t size) { - parsePlan(data, size, {-1, -1, -1}); - } - - /// Parse and cache the plan. - /// Return true if parsed successfully. - void parsePlan(const uint8_t* data, int32_t size, SparkTaskInfo taskInfo) { - taskInfo_ = taskInfo; -#ifdef GLUTEN_PRINT_DEBUG - try { - auto jsonPlan = substraitFromPbToJson("Plan", data, size); - DEBUG_OUT << std::string(50, '#') << " received substrait::Plan:" << std::endl; - DEBUG_OUT << "Task stageId: " << taskInfo_.stageId << ", partitionId: " << taskInfo_.partitionId - << ", taskId: " << taskInfo_.taskId << "; " << jsonPlan << std::endl; - } catch (const std::exception& e) { - std::cerr << "Error converting Substrait plan to JSON: " << e.what() << std::endl; - } -#endif - GLUTEN_CHECK(parseProtobuf(data, size, &substraitPlan_) == true, "Parse substrait plan failed"); - } - - // Just for benchmark - ::substrait::Plan& getPlan() { - return substraitPlan_; - } - - virtual MemoryManager* createMemoryManager( - const std::string& name, - std::shared_ptr, - std::unique_ptr) = 0; - - /// This function is used to create certain converter from the format used by - /// the backend to Spark unsafe row. - virtual ResourceHandle createColumnar2RowConverter(MemoryManager* memoryManager) = 0; - virtual std::shared_ptr getColumnar2RowConverter(ResourceHandle) = 0; - virtual void releaseColumnar2RowConverter(ResourceHandle) = 0; - - virtual ResourceHandle createRow2ColumnarConverter(MemoryManager* memoryManager, struct ArrowSchema* cSchema) = 0; - virtual std::shared_ptr getRow2ColumnarConverter(ResourceHandle) = 0; - virtual void releaseRow2ColumnarConverter(ResourceHandle) = 0; - - virtual ResourceHandle createShuffleWriter( - int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, - MemoryManager* memoryManager) = 0; - virtual std::shared_ptr getShuffleWriter(ResourceHandle) = 0; - virtual void releaseShuffleWriter(ResourceHandle) = 0; - - virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) = 0; - - virtual ResourceHandle createDatasource( - const std::string& filePath, - MemoryManager* memoryManager, - std::shared_ptr schema) = 0; - virtual std::shared_ptr getDatasource(ResourceHandle) = 0; - virtual void releaseDatasource(ResourceHandle) = 0; - - virtual ResourceHandle createShuffleReader( - std::shared_ptr schema, - ReaderOptions options, - arrow::MemoryPool* pool, - MemoryManager* memoryManager) = 0; - virtual std::shared_ptr getShuffleReader(ResourceHandle) = 0; - virtual void releaseShuffleReader(ResourceHandle) = 0; - - virtual ResourceHandle createColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) = 0; - // TODO: separate serializer and deserializer then remove this method. - virtual std::unique_ptr createTempColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) = 0; - virtual std::shared_ptr getColumnarBatchSerializer(ResourceHandle) = 0; - virtual void releaseColumnarBatchSerializer(ResourceHandle) = 0; - - const std::unordered_map& getConfMap() { - return confMap_; - } - - SparkTaskInfo getSparkTaskInfo() { - return taskInfo_; - } - - protected: - ::substrait::Plan substraitPlan_; - SparkTaskInfo taskInfo_; - // Session conf map - const std::unordered_map confMap_; -}; -} // namespace gluten diff --git a/cpp/core/compute/ResultIterator.cc b/cpp/core/compute/ResultIterator.cc index ba0eabde1fbf..8ba215bd275b 100644 --- a/cpp/core/compute/ResultIterator.cc +++ b/cpp/core/compute/ResultIterator.cc @@ -16,13 +16,13 @@ */ #include "ResultIterator.h" -#include "ExecutionCtx.h" +#include "Runtime.h" namespace gluten { Metrics* ResultIterator::getMetrics() { - if (executionCtx_) { - return executionCtx_->getMetrics(getInputIter(), exportNanos_); + if (runtime_) { + return runtime_->getMetrics(getInputIter(), exportNanos_); } return nullptr; } diff --git a/cpp/core/compute/ResultIterator.h b/cpp/core/compute/ResultIterator.h index 79082fd7f593..041f76b22d56 100644 --- a/cpp/core/compute/ResultIterator.h +++ b/cpp/core/compute/ResultIterator.h @@ -23,14 +23,14 @@ namespace gluten { -class ExecutionCtx; +class Runtime; // FIXME the code is tightly coupled with Velox plan execution. Should cleanup the abstraction for uses from // other places. class ResultIterator { public: - explicit ResultIterator(std::unique_ptr iter, ExecutionCtx* executionCtx = nullptr) - : iter_(std::move(iter)), next_(nullptr), executionCtx_(executionCtx) {} + explicit ResultIterator(std::unique_ptr iter, Runtime* runtime = nullptr) + : iter_(std::move(iter)), next_(nullptr), runtime_(runtime) {} // copy constructor and copy assignment (deleted) ResultIterator(const ResultIterator& in) = delete; @@ -86,7 +86,7 @@ class ResultIterator { std::unique_ptr iter_; std::shared_ptr next_; - ExecutionCtx* executionCtx_; + Runtime* runtime_; int64_t exportNanos_; }; diff --git a/cpp/core/compute/ExecutionCtx.cc b/cpp/core/compute/Runtime.cc similarity index 56% rename from cpp/core/compute/ExecutionCtx.cc rename to cpp/core/compute/Runtime.cc index bf56592580b5..387da1a0a897 100644 --- a/cpp/core/compute/ExecutionCtx.cc +++ b/cpp/core/compute/Runtime.cc @@ -15,7 +15,7 @@ * limitations under the License. */ -#include "ExecutionCtx.h" +#include "Runtime.h" #include "utils/Print.h" namespace gluten { @@ -23,48 +23,46 @@ namespace gluten { namespace { class FactoryRegistry { public: - void registerFactory(const std::string& kind, ExecutionCtx::Factory factory) { + void registerFactory(const std::string& kind, Runtime::Factory factory) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) == map_.end(), "ExecutionCtx factory already registered for " + kind); + GLUTEN_CHECK(map_.find(kind) == map_.end(), "Runtime factory already registered for " + kind); map_[kind] = std::move(factory); } - ExecutionCtx::Factory& getFactory(const std::string& kind) { + Runtime::Factory& getFactory(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "ExecutionCtx factory not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Runtime factory not registered for " + kind); return map_[kind]; } bool unregisterFactory(const std::string& kind) { std::lock_guard l(mutex_); - GLUTEN_CHECK(map_.find(kind) != map_.end(), "ExecutionCtx factory not registered for " + kind); + GLUTEN_CHECK(map_.find(kind) != map_.end(), "Runtime factory not registered for " + kind); return map_.erase(kind); } private: std::mutex mutex_; - std::unordered_map map_; + std::unordered_map map_; }; -FactoryRegistry& executionCtxFactories() { +FactoryRegistry& runtimeFactories() { static FactoryRegistry registry; return registry; } } // namespace -void ExecutionCtx::registerFactory(const std::string& kind, ExecutionCtx::Factory factory) { - executionCtxFactories().registerFactory(kind, std::move(factory)); +void Runtime::registerFactory(const std::string& kind, Runtime::Factory factory) { + runtimeFactories().registerFactory(kind, std::move(factory)); } -ExecutionCtx* ExecutionCtx::create( - const std::string& kind, - const std::unordered_map& sessionConf) { - auto& factory = executionCtxFactories().getFactory(kind); +Runtime* Runtime::create(const std::string& kind, const std::unordered_map& sessionConf) { + auto& factory = runtimeFactories().getFactory(kind); return factory(sessionConf); } -void ExecutionCtx::release(ExecutionCtx* executionCtx) { - delete executionCtx; +void Runtime::release(Runtime* runtime) { + delete runtime; } } // namespace gluten diff --git a/cpp/core/compute/Runtime.h b/cpp/core/compute/Runtime.h new file mode 100644 index 000000000000..f188e24203d1 --- /dev/null +++ b/cpp/core/compute/Runtime.h @@ -0,0 +1,143 @@ +/* + * 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 "compute/ProtobufUtils.h" +#include "compute/ResultIterator.h" +#include "memory/ArrowMemoryPool.h" +#include "memory/ColumnarBatch.h" +#include "memory/MemoryManager.h" +#include "operators/c2r/ColumnarToRow.h" +#include "operators/r2c/RowToColumnar.h" +#include "operators/serializer/ColumnarBatchSerializer.h" +#include "operators/writer/Datasource.h" +#include "shuffle/ShuffleReader.h" +#include "shuffle/ShuffleWriter.h" +#include "substrait/plan.pb.h" +#include "utils/DebugOut.h" +#include "utils/ObjectStore.h" + +namespace gluten { + +class ResultIterator; + +struct SparkTaskInfo { + int32_t stageId{0}; + int32_t partitionId{0}; + // Same as TID. + int64_t taskId{0}; + + std::string toString() const { + return "[Stage: " + std::to_string(stageId) + " TID: " + std::to_string(taskId) + "]"; + } + + friend std::ostream& operator<<(std::ostream& os, const SparkTaskInfo& taskInfo) { + os << "[Stage: " << taskInfo.stageId << " TID: " << taskInfo.taskId << "]"; + return os; + } +}; + +class Runtime : public std::enable_shared_from_this { + public: + using Factory = std::function&)>; + static void registerFactory(const std::string& kind, Factory factory); + static Runtime* create(const std::string& kind, const std::unordered_map& sessionConf = {}); + static void release(Runtime*); + + Runtime() = default; + Runtime(const std::unordered_map& confMap) : confMap_(confMap) {} + virtual ~Runtime() = default; + + /// Parse and cache the plan. + /// Return true if parsed successfully. + virtual void parsePlan(const uint8_t* data, int32_t size, SparkTaskInfo taskInfo) = 0; + + virtual std::string planString(bool details, const std::unordered_map& sessionConf) = 0; + + // Just for benchmark + ::substrait::Plan& getPlan() { + return substraitPlan_; + } + + virtual std::shared_ptr createResultIterator( + MemoryManager* memoryManager, + const std::string& spillDir, + const std::vector>& inputs, + const std::unordered_map& sessionConf) = 0; + + virtual std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) = 0; + + virtual std::shared_ptr + select(MemoryManager*, std::shared_ptr, std::vector) = 0; + + virtual MemoryManager* createMemoryManager( + const std::string& name, + std::shared_ptr, + std::unique_ptr) = 0; + + /// This function is used to create certain converter from the format used by + /// the backend to Spark unsafe row. + virtual std::shared_ptr createColumnar2RowConverter(MemoryManager* memoryManager) = 0; + + virtual std::shared_ptr createRow2ColumnarConverter( + MemoryManager* memoryManager, + struct ArrowSchema* cSchema) = 0; + + virtual std::shared_ptr createShuffleWriter( + int numPartitions, + std::shared_ptr partitionWriterCreator, + const ShuffleWriterOptions& options, + MemoryManager* memoryManager) = 0; + virtual Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) = 0; + + virtual std::shared_ptr createDatasource( + const std::string& filePath, + MemoryManager* memoryManager, + std::shared_ptr schema) = 0; + + virtual std::shared_ptr createShuffleReader( + std::shared_ptr schema, + ShuffleReaderOptions options, + arrow::MemoryPool* pool, + MemoryManager* memoryManager) = 0; + + virtual std::unique_ptr createColumnarBatchSerializer( + MemoryManager* memoryManager, + arrow::MemoryPool* arrowPool, + struct ArrowSchema* cSchema) = 0; + + const std::unordered_map& getConfMap() { + return confMap_; + } + + SparkTaskInfo getSparkTaskInfo() { + return taskInfo_; + } + + ObjectStore* objectStore() { + return objStore_.get(); + } + + protected: + std::unique_ptr objStore_ = ObjectStore::create(); + ::substrait::Plan substraitPlan_; + SparkTaskInfo taskInfo_; + // Session conf map + const std::unordered_map confMap_; +}; +} // namespace gluten diff --git a/cpp/core/config/GlutenConfig.cc b/cpp/core/config/GlutenConfig.cc index 233053c817e7..21f8705e0cf5 100644 --- a/cpp/core/config/GlutenConfig.cc +++ b/cpp/core/config/GlutenConfig.cc @@ -34,4 +34,13 @@ std::unordered_map parseConfMap(JNIEnv* env, jbyteArra return sparkConfs; } + +std::string printConfig(const std::unordered_map& conf) { + std::ostringstream oss; + oss << std::endl; + for (auto& [k, v] : conf) { + oss << " [" << k << ", " << v << "]\n"; + } + return oss.str(); +} } // namespace gluten diff --git a/cpp/core/config/GlutenConfig.h b/cpp/core/config/GlutenConfig.h index 4c139223fb8c..c96dc2844c70 100644 --- a/cpp/core/config/GlutenConfig.h +++ b/cpp/core/config/GlutenConfig.h @@ -24,11 +24,14 @@ namespace gluten { // store configurations that are general to all backend types +const std::string kDebugModeEnabled = "spark.gluten.sql.debug"; const std::string kGlutenSaveDir = "spark.gluten.saveDir"; const std::string kCaseSensitive = "spark.sql.caseSensitive"; +const std::string kLegacySize = "spark.sql.legacy.sizeOfNull"; + const std::string kSessionTimezone = "spark.sql.session.timeZone"; const std::string kDefaultSessionTimezone = "spark.gluten.sql.session.timeZone.default"; @@ -43,6 +46,9 @@ const std::string kParquetBlockSize = "parquet.block.size"; const std::string kParquetBlockRows = "parquet.block.rows"; +const std::string kParquetGzipWindowSize = "parquet.gzip.windowSize"; +const std::string kGzipWindowSize4k = "4096"; + const std::string kParquetCompressionCodec = "spark.sql.parquet.compression.codec"; const std::string kUGIUserName = "spark.gluten.ugi.username"; @@ -54,4 +60,6 @@ const std::string kQatBackendName = "qat"; const std::string kIaaBackendName = "iaa"; std::unordered_map parseConfMap(JNIEnv* env, jbyteArray configArray); + +std::string printConfig(const std::unordered_map& conf); } // namespace gluten diff --git a/cpp/core/jni/JniCommon.cc b/cpp/core/jni/JniCommon.cc index 96a52137f4a9..3d3ad6d4ed43 100644 --- a/cpp/core/jni/JniCommon.cc +++ b/cpp/core/jni/JniCommon.cc @@ -32,14 +32,14 @@ void gluten::JniCommonState::assertInitialized() { } } -jmethodID gluten::JniCommonState::executionCtxAwareCtxHandle() { +jmethodID gluten::JniCommonState::runtimeAwareCtxHandle() { assertInitialized(); - return executionCtxAwareCtxHandle_; + return runtimeAwareCtxHandle_; } void gluten::JniCommonState::initialize(JNIEnv* env) { - executionCtxAwareClass_ = createGlobalClassReference(env, "Lio/glutenproject/exec/ExecutionCtxAware;"); - executionCtxAwareCtxHandle_ = getMethodIdOrError(env, executionCtxAwareClass_, "ctxHandle", "()J"); + runtimeAwareClass_ = createGlobalClassReference(env, "Lio/glutenproject/exec/RuntimeAware;"); + runtimeAwareCtxHandle_ = getMethodIdOrError(env, runtimeAwareClass_, "handle", "()J"); JavaVM* vm; if (env->GetJavaVM(&vm) != JNI_OK) { throw gluten::GlutenException("Unable to get JavaVM instance"); @@ -54,14 +54,14 @@ void gluten::JniCommonState::close() { } JNIEnv* env; attachCurrentThreadAsDaemonOrThrow(vm_, &env); - env->DeleteGlobalRef(executionCtxAwareClass_); + env->DeleteGlobalRef(runtimeAwareClass_); closed_ = true; } -gluten::ExecutionCtx* gluten::getExecutionCtx(JNIEnv* env, jobject executionCtxAware) { - int64_t ctxHandle = env->CallLongMethod(executionCtxAware, getJniCommonState()->executionCtxAwareCtxHandle()); +gluten::Runtime* gluten::getRuntime(JNIEnv* env, jobject runtimeAware) { + int64_t ctxHandle = env->CallLongMethod(runtimeAware, getJniCommonState()->runtimeAwareCtxHandle()); checkException(env); - auto ctx = reinterpret_cast(ctxHandle); + auto ctx = reinterpret_cast(ctxHandle); GLUTEN_CHECK(ctx != nullptr, "FATAL: resource instance should not be null."); return ctx; } diff --git a/cpp/core/jni/JniCommon.h b/cpp/core/jni/JniCommon.h index 01444ceae7e9..265382aa2d20 100644 --- a/cpp/core/jni/JniCommon.h +++ b/cpp/core/jni/JniCommon.h @@ -22,13 +22,13 @@ #include #include -#include "compute/ExecutionCtx.h" #include "compute/ProtobufUtils.h" +#include "compute/Runtime.h" #include "config/GlutenConfig.h" #include "memory/AllocationListener.h" #include "shuffle/rss/RssClient.h" +#include "utils/Compression.h" #include "utils/DebugOut.h" -#include "utils/compression.h" #include "utils/exception.h" static jint jniVersion = JNI_VERSION_1_8; @@ -132,13 +132,13 @@ class JniCommonState { void close(); - jmethodID executionCtxAwareCtxHandle(); + jmethodID runtimeAwareCtxHandle(); private: void initialize(JNIEnv* env); - jclass executionCtxAwareClass_; - jmethodID executionCtxAwareCtxHandle_; + jclass runtimeAwareClass_; + jmethodID runtimeAwareCtxHandle_; JavaVM* vm_; bool initialized_{false}; @@ -151,7 +151,7 @@ inline JniCommonState* getJniCommonState() { return &jniCommonState; } -ExecutionCtx* getExecutionCtx(JNIEnv* env, jobject executionCtxAware); +Runtime* getRuntime(JNIEnv* env, jobject runtimeAware); } // namespace gluten // TODO: Move the static functions to namespace gluten @@ -170,18 +170,15 @@ static inline arrow::Compression::type getCompressionType(JNIEnv* env, jstring c if (codecJstr == NULL) { return arrow::Compression::UNCOMPRESSED; } - auto codecU = env->GetStringUTFChars(codecJstr, JNI_FALSE); + auto codec = env->GetStringUTFChars(codecJstr, JNI_FALSE); - std::string codecL; - std::transform(codecU, codecU + std::strlen(codecU), std::back_inserter(codecL), ::tolower); + // Convert codec string into lowercase. + std::string codecLower; + std::transform(codec, codec + std::strlen(codec), std::back_inserter(codecLower), ::tolower); + GLUTEN_ASSIGN_OR_THROW(auto compressionType, arrow::util::Codec::GetCompressionType(codecLower)); - GLUTEN_ASSIGN_OR_THROW(auto compression_type, arrow::util::Codec::GetCompressionType(codecL)); - - if (compression_type == arrow::Compression::LZ4) { - compression_type = arrow::Compression::LZ4_FRAME; - } - env->ReleaseStringUTFChars(codecJstr, codecU); - return compression_type; + env->ReleaseStringUTFChars(codecJstr, codec); + return compressionType; } static inline gluten::CodecBackend getCodecBackend(JNIEnv* env, jstring codecJstr) { diff --git a/cpp/core/jni/JniWrapper.cc b/cpp/core/jni/JniWrapper.cc index b8dcc9316027..eefe8b867a72 100644 --- a/cpp/core/jni/JniWrapper.cc +++ b/cpp/core/jni/JniWrapper.cc @@ -19,8 +19,8 @@ #include #include -#include "compute/ExecutionCtx.h" #include "compute/ProtobufUtils.h" +#include "compute/Runtime.h" #include "config/GlutenConfig.h" #include "jni/JniCommon.h" #include "jni/JniError.h" @@ -32,6 +32,7 @@ #include "operators/serializer/ColumnarBatchSerializer.h" #include "shuffle/LocalPartitionWriter.h" #include "shuffle/PartitionWriterCreator.h" +#include "shuffle/Partitioning.h" #include "shuffle/ShuffleReader.h" #include "shuffle/ShuffleWriter.h" #include "shuffle/Utils.h" @@ -40,8 +41,6 @@ using namespace gluten; -static jclass serializableObjBuilderClass; - static jclass javaReservationListenerClass; static jmethodID reserveMemoryMethod; @@ -156,9 +155,9 @@ class JniColumnarBatchIterator : public ColumnarBatchIterator { explicit JniColumnarBatchIterator( JNIEnv* env, jobject jColumnarBatchItr, - ExecutionCtx* executionCtx, + Runtime* runtime, std::shared_ptr writer) - : executionCtx_(executionCtx), writer_(writer) { + : runtime_(runtime), writer_(writer) { // IMPORTANT: DO NOT USE LOCAL REF IN DIFFERENT THREAD if (env->GetJavaVM(&vm_) != JNI_OK) { std::string errorMessage = "Unable to get JavaVM instance"; @@ -191,7 +190,7 @@ class JniColumnarBatchIterator : public ColumnarBatchIterator { checkException(env); jlong handle = env->CallLongMethod(jColumnarBatchItr_, serializedColumnarBatchIteratorNext); checkException(env); - auto batch = executionCtx_->getBatch(handle); + auto batch = runtime_->objectStore()->retrieve(handle); if (writer_ != nullptr) { // save snapshot of the batch to file std::shared_ptr schema = batch->exportArrowSchema(); @@ -206,16 +205,16 @@ class JniColumnarBatchIterator : public ColumnarBatchIterator { private: JavaVM* vm_; jobject jColumnarBatchItr_; - ExecutionCtx* executionCtx_; + Runtime* runtime_; std::shared_ptr writer_; }; std::unique_ptr makeJniColumnarBatchIterator( JNIEnv* env, jobject jColumnarBatchItr, - ExecutionCtx* executionCtx, + Runtime* runtime, std::shared_ptr writer) { - return std::make_unique(env, jColumnarBatchItr, executionCtx, writer); + return std::make_unique(env, jColumnarBatchItr, runtime, writer); } template @@ -237,9 +236,6 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { gluten::getJniCommonState()->ensureInitialized(env); gluten::getJniErrorState()->ensureInitialized(env); - serializableObjBuilderClass = - createGlobalClassReferenceOrError(env, "Lio/glutenproject/vectorized/NativeSerializableObject;"); - byteArrayClass = createGlobalClassReferenceOrError(env, "[B"); jniByteInputStreamClass = createGlobalClassReferenceOrError(env, "Lio/glutenproject/vectorized/JniByteInputStream;"); @@ -298,7 +294,6 @@ jint JNI_OnLoad(JavaVM* vm, void* reserved) { void JNI_OnUnload(JavaVM* vm, void* reserved) { JNIEnv* env; vm->GetEnv(reinterpret_cast(&env), jniVersion); - env->DeleteGlobalRef(serializableObjBuilderClass); env->DeleteGlobalRef(jniByteInputStreamClass); env->DeleteGlobalRef(splitResultClass); env->DeleteGlobalRef(columnarBatchSerializeResultClass); @@ -312,7 +307,7 @@ void JNI_OnUnload(JavaVM* vm, void* reserved) { gluten::getJniCommonState()->close(); } -JNIEXPORT jlong JNICALL Java_io_glutenproject_exec_ExecutionCtxJniWrapper_createExecutionCtx( // NOLINT +JNIEXPORT jlong JNICALL Java_io_glutenproject_exec_RuntimeJniWrapper_createRuntime( // NOLINT JNIEnv* env, jclass, jstring jbackendType, @@ -320,22 +315,40 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_exec_ExecutionCtxJniWrapper_create JNI_METHOD_START auto backendType = jStringToCString(env, jbackendType); auto sparkConf = gluten::parseConfMap(env, sessionConf); - auto executionCtx = gluten::ExecutionCtx::create(backendType, sparkConf); - return reinterpret_cast(executionCtx); + auto runtime = gluten::Runtime::create(backendType, sparkConf); + return reinterpret_cast(runtime); JNI_METHOD_END(kInvalidResourceHandle) } -JNIEXPORT void JNICALL Java_io_glutenproject_exec_ExecutionCtxJniWrapper_releaseExecutionCtx( // NOLINT +JNIEXPORT void JNICALL Java_io_glutenproject_exec_RuntimeJniWrapper_releaseRuntime( // NOLINT JNIEnv* env, jclass, jlong ctxHandle) { JNI_METHOD_START - auto executionCtx = jniCastOrThrow(ctxHandle); + auto runtime = jniCastOrThrow(ctxHandle); - gluten::ExecutionCtx::release(executionCtx); + gluten::Runtime::release(runtime); JNI_METHOD_END() } +JNIEXPORT jstring JNICALL Java_io_glutenproject_vectorized_PlanEvaluatorJniWrapper_nativePlanString( // NOLINT + JNIEnv* env, + jobject wrapper, + jbyteArray planArray, + jboolean details) { + JNI_METHOD_START + + auto planData = reinterpret_cast(env->GetByteArrayElements(planArray, 0)); + auto planSize = env->GetArrayLength(planArray); + auto ctx = gluten::getRuntime(env, wrapper); + ctx->parsePlan(planData, planSize, {}); + auto& conf = ctx->getConfMap(); + auto planString = ctx->planString(details, conf); + return env->NewStringUTF(planString.c_str()); + + JNI_METHOD_END(nullptr) +} + JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWithIterator( // NOLINT JNIEnv* env, @@ -350,7 +363,7 @@ Java_io_glutenproject_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWithI jstring spillDir) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); auto spillDirStr = jStringToCString(env, spillDir); @@ -382,7 +395,7 @@ Java_io_glutenproject_vectorized_PlanEvaluatorJniWrapper_nativeCreateKernelWithI inputIters.push_back(std::move(resultIter)); } - return ctx->createResultIterator(memoryManager, spillDirStr, inputIters, conf); + return ctx->objectStore()->save(ctx->createResultIterator(memoryManager, spillDirStr, inputIters, conf)); JNI_METHOD_END(kInvalidResourceHandle) } @@ -391,11 +404,11 @@ JNIEXPORT jboolean JNICALL Java_io_glutenproject_vectorized_ColumnarBatchOutIter jobject wrapper, jlong iterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto iter = ctx->getResultIterator(iterHandle); + auto iter = ctx->objectStore()->retrieve(iterHandle); if (iter == nullptr) { - std::string errorMessage = "faked to get batch iterator"; + std::string errorMessage = "failed to get batch iterator"; throw gluten::GlutenException(errorMessage); } return iter->hasNext(); @@ -407,15 +420,15 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ColumnarBatchOutIterato jobject wrapper, jlong iterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto iter = ctx->getResultIterator(iterHandle); + auto iter = ctx->objectStore()->retrieve(iterHandle); if (!iter->hasNext()) { return kInvalidResourceHandle; } std::shared_ptr batch = iter->next(); - auto batchHandle = ctx->addBatch(batch); + auto batchHandle = ctx->objectStore()->save(batch); iter->setExportNanos(batch->getExportNanos()); return batchHandle; @@ -427,9 +440,9 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_vectorized_ColumnarBatchOutItera jobject wrapper, jlong iterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto iter = ctx->getResultIterator(iterHandle); + auto iter = ctx->objectStore()->retrieve(iterHandle); auto metrics = iter->getMetrics(); unsigned int numMetrics = 0; if (metrics) { @@ -483,9 +496,9 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ColumnarBatchOutIterato jlong iterHandle, jlong size) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto it = ctx->getResultIterator(iterHandle); + auto it = ctx->objectStore()->retrieve(iterHandle); return it->spillFixedSize(size); JNI_METHOD_END(kInvalidResourceHandle) } @@ -495,9 +508,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ColumnarBatchOutIterator jobject wrapper, jlong iterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - ctx->releaseResultIterator(iterHandle); + ctx->objectStore()->release(iterHandle); JNI_METHOD_END() } @@ -507,11 +520,11 @@ Java_io_glutenproject_vectorized_NativeColumnarToRowJniWrapper_nativeColumnarToR jobject wrapper, jlong memoryManagerHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); // Convert the native batch to Spark unsafe row. - return ctx->createColumnar2RowConverter(memoryManager); + return ctx->objectStore()->save(ctx->createColumnar2RowConverter(memoryManager)); JNI_METHOD_END(kInvalidResourceHandle) } @@ -522,9 +535,9 @@ Java_io_glutenproject_vectorized_NativeColumnarToRowJniWrapper_nativeColumnarToR jlong batchHandle, jlong c2rHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto columnarToRowConverter = ctx->getColumnar2RowConverter(c2rHandle); - auto cb = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto columnarToRowConverter = ctx->objectStore()->retrieve(c2rHandle); + auto cb = ctx->objectStore()->retrieve(batchHandle); columnarToRowConverter->convert(cb); const auto& offsets = columnarToRowConverter->getOffsets(); @@ -551,9 +564,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_NativeColumnarToRowJniWr jobject wrapper, jlong c2rHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - ctx->releaseColumnar2RowConverter(c2rHandle); + ctx->objectStore()->release(c2rHandle); JNI_METHOD_END() } @@ -563,10 +576,11 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_NativeRowToColumnarJniW jlong cSchema, jlong memoryManagerHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); - return ctx->createRow2ColumnarConverter(memoryManager, reinterpret_cast(cSchema)); + return ctx->objectStore()->save( + ctx->createRow2ColumnarConverter(memoryManager, reinterpret_cast(cSchema))); JNI_METHOD_END(kInvalidResourceHandle) } @@ -578,7 +592,7 @@ Java_io_glutenproject_vectorized_NativeRowToColumnarJniWrapper_nativeConvertRowT jlongArray rowLength, jlong memoryAddress) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); if (rowLength == nullptr) { throw gluten::GlutenException("Native convert row to columnar: buf_addrs can't be null"); @@ -587,10 +601,10 @@ Java_io_glutenproject_vectorized_NativeRowToColumnarJniWrapper_nativeConvertRowT jlong* inRowLength = env->GetLongArrayElements(rowLength, nullptr); uint8_t* address = reinterpret_cast(memoryAddress); - auto converter = ctx->getRow2ColumnarConverter(r2cHandle); + auto converter = ctx->objectStore()->retrieve(r2cHandle); auto cb = converter->convert(numRows, reinterpret_cast(inRowLength), address); env->ReleaseLongArrayElements(rowLength, inRowLength, JNI_ABORT); - return ctx->addBatch(cb); + return ctx->objectStore()->save(cb); JNI_METHOD_END(kInvalidResourceHandle) } @@ -599,9 +613,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_NativeRowToColumnarJniWr jobject wrapper, jlong r2cHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - ctx->releaseRow2ColumnarConverter(r2cHandle); + ctx->objectStore()->release(r2cHandle); JNI_METHOD_END() } @@ -610,8 +624,8 @@ JNIEXPORT jstring JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWr jobject wrapper, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); return env->NewStringUTF(batch->getType().c_str()); JNI_METHOD_END(nullptr) } @@ -621,8 +635,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jobject wrapper, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); return batch->numBytes(); JNI_METHOD_END(kInvalidResourceHandle) } @@ -632,8 +646,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jobject wrapper, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); return batch->numColumns(); JNI_METHOD_END(kInvalidResourceHandle) } @@ -643,8 +657,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jobject wrapper, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); return batch->numRows(); JNI_METHOD_END(kInvalidResourceHandle) } @@ -654,7 +668,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jobject wrapper, jlongArray batchHandles) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); int handleCount = env->GetArrayLength(batchHandles); jlong* handleArray = env->GetLongArrayElements(batchHandles, nullptr); @@ -662,12 +676,12 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap std::vector> batches; for (int i = 0; i < handleCount; ++i) { jlong handle = handleArray[i]; - auto batch = ctx->getBatch(handle); + auto batch = ctx->objectStore()->retrieve(handle); batches.push_back(batch); } auto newBatch = CompositeColumnarBatch::create(std::move(batches)); env->ReleaseLongArrayElements(batchHandles, handleArray, JNI_ABORT); - return ctx->addBatch(newBatch); + return ctx->objectStore()->save(newBatch); JNI_METHOD_END(kInvalidResourceHandle) } @@ -678,8 +692,8 @@ JNIEXPORT void JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrapp jlong cSchema, jlong cArray) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); std::shared_ptr exportedSchema = batch->exportArrowSchema(); std::shared_ptr exportedArray = batch->exportArrowArray(); ArrowSchemaMove(exportedSchema.get(), reinterpret_cast(cSchema)); @@ -693,7 +707,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jlong cSchema, jlong cArray) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); std::unique_ptr targetSchema = std::make_unique(); std::unique_ptr targetArray = std::make_unique(); @@ -703,7 +717,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap ArrowSchemaMove(arrowSchema, targetSchema.get()); std::shared_ptr batch = std::make_shared(std::move(targetSchema), std::move(targetArray)); - return ctx->addBatch(batch); + return ctx->objectStore()->save(batch); JNI_METHOD_END(kInvalidResourceHandle) } @@ -712,8 +726,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jobject wrapper, jint numRows) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - return ctx->createOrGetEmptySchemaBatch(static_cast(numRows)); + auto ctx = gluten::getRuntime(env, wrapper); + return ctx->objectStore()->save(ctx->createOrGetEmptySchemaBatch(static_cast(numRows))); JNI_METHOD_END(kInvalidResourceHandle) } @@ -724,7 +738,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap jlong batchHandle, jintArray jcolumnIndices) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); int* tmp = env->GetIntArrayElements(jcolumnIndices, nullptr); @@ -735,7 +749,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrap } env->ReleaseIntArrayElements(jcolumnIndices, tmp, JNI_ABORT); - return ctx->select(memoryManager, batchHandle, std::move(columnIndices)); + return ctx->objectStore()->save( + ctx->select(memoryManager, ctx->objectStore()->retrieve(batchHandle), std::move(columnIndices))); JNI_METHOD_END(kInvalidResourceHandle) } @@ -744,8 +759,8 @@ JNIEXPORT void JNICALL Java_io_glutenproject_columnarbatch_ColumnarBatchJniWrapp jobject wrapper, jlong batchHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - ctx->releaseBatch(batchHandle); + auto ctx = gluten::getRuntime(env, wrapper); + ctx->objectStore()->release(batchHandle); JNI_METHOD_END() } @@ -772,24 +787,24 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper jobject partitionPusher, jstring partitionWriterTypeJstr) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); if (partitioningNameJstr == nullptr) { throw gluten::GlutenException(std::string("Short partitioning name can't be null")); return kInvalidResourceHandle; } + auto shuffleWriterOptions = ShuffleWriterOptions::defaults(); + auto partitioningName = jStringToCString(env, partitioningNameJstr); + shuffleWriterOptions.partitioning = gluten::toPartitioning(partitioningName); - auto shuffleWriterOptions = ShuffleWriterOptions::defaults(); - shuffleWriterOptions.partitioning_name = partitioningName; - shuffleWriterOptions.buffered_write = true; if (bufferSize > 0) { shuffleWriterOptions.buffer_size = bufferSize; } + shuffleWriterOptions.compression_type = getCompressionType(env, codecJstr); if (codecJstr != NULL) { - shuffleWriterOptions.compression_type = getCompressionType(env, codecJstr); shuffleWriterOptions.codec_backend = getCodecBackend(env, codecBackendJstr); shuffleWriterOptions.compression_mode = getCompressionMode(env, compressionModeJstr); } @@ -810,7 +825,7 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper } shuffleWriterOptions.task_attempt_id = (int64_t)taskAttemptId; - shuffleWriterOptions.buffer_compress_threshold = bufferCompressThreshold; + shuffleWriterOptions.compression_threshold = bufferCompressThreshold; auto partitionWriterTypeC = env->GetStringUTFChars(partitionWriterTypeJstr, JNI_FALSE); auto partitionWriterType = std::string(partitionWriterTypeC); @@ -839,8 +854,9 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper env->ReleaseStringUTFChars(dataFileJstr, dataFileC); auto localDirs = env->GetStringUTFChars(localDirsJstr, JNI_FALSE); - setenv(gluten::kGlutenSparkLocalDirs.c_str(), localDirs, 1); + shuffleWriterOptions.local_dirs = std::string(localDirs); env->ReleaseStringUTFChars(localDirsJstr, localDirs); + partitionWriterCreator = std::make_shared(); } else if (partitionWriterType == "celeborn") { shuffleWriterOptions.partition_writer_type = PartitionWriterType::kCeleborn; @@ -862,8 +878,8 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper throw gluten::GlutenException("Unrecognizable partition writer type: " + partitionWriterType); } - return ctx->createShuffleWriter( - numPartitions, std::move(partitionWriterCreator), std::move(shuffleWriterOptions), memoryManager); + return ctx->objectStore()->save(ctx->createShuffleWriter( + numPartitions, std::move(partitionWriterCreator), std::move(shuffleWriterOptions), memoryManager)); JNI_METHOD_END(kInvalidResourceHandle) } @@ -874,9 +890,9 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper jlong size, jboolean callBySelf) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto shuffleWriter = ctx->getShuffleWriter(shuffleWriterHandle); + auto shuffleWriter = ctx->objectStore()->retrieve(shuffleWriterHandle); if (!shuffleWriter) { std::string errorMessage = "Invalid shuffle writer handle " + std::to_string(shuffleWriterHandle); throw gluten::GlutenException(errorMessage); @@ -896,16 +912,16 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper jlong batchHandle, jlong memLimit) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto shuffleWriter = ctx->getShuffleWriter(shuffleWriterHandle); + auto shuffleWriter = ctx->objectStore()->retrieve(shuffleWriterHandle); if (!shuffleWriter) { std::string errorMessage = "Invalid shuffle writer handle " + std::to_string(shuffleWriterHandle); throw gluten::GlutenException(errorMessage); } // The column batch maybe VeloxColumnBatch or ArrowCStructColumnarBatch(FallbackRangeShuffleWriter) - auto batch = ctx->getBatch(batchHandle); + auto batch = ctx->objectStore()->retrieve(batchHandle); auto numBytes = batch->numBytes(); gluten::arrowAssertOkOrThrow(shuffleWriter->split(batch, memLimit), "Native split: shuffle writer split failed"); return numBytes; @@ -917,9 +933,9 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapp jobject wrapper, jlong shuffleWriterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto shuffleWriter = ctx->getShuffleWriter(shuffleWriterHandle); + auto shuffleWriter = ctx->objectStore()->retrieve(shuffleWriterHandle); if (!shuffleWriter) { std::string errorMessage = "Invalid shuffle writer handle " + std::to_string(shuffleWriterHandle); throw gluten::GlutenException(errorMessage); @@ -959,9 +975,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ShuffleWriterJniWrapper_ jobject wrapper, jlong shuffleWriterHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - ctx->releaseShuffleWriter(shuffleWriterHandle); + ctx->objectStore()->release(shuffleWriterHandle); JNI_METHOD_END() } @@ -984,25 +1000,23 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper jlong cSchema, jlong memoryManagerHandle, jstring compressionType, - jstring compressionBackend, - jstring compressionMode) { + jstring compressionBackend) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); auto pool = memoryManager->getArrowMemoryPool(); - ReaderOptions options = ReaderOptions::defaults(); + ShuffleReaderOptions options = ShuffleReaderOptions::defaults(); options.ipc_read_options.memory_pool = pool; options.ipc_read_options.use_threads = false; + options.compression_type = getCompressionType(env, compressionType); if (compressionType != nullptr) { - options.compression_type = getCompressionType(env, compressionType); options.codec_backend = getCodecBackend(env, compressionBackend); - options.compression_mode = getCompressionMode(env, compressionMode); } std::shared_ptr schema = gluten::arrowGetOrThrow(arrow::ImportSchema(reinterpret_cast(cSchema))); - return ctx->createShuffleReader(schema, options, pool, memoryManager); + return ctx->objectStore()->save(ctx->createShuffleReader(schema, options, pool, memoryManager)); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1012,12 +1026,12 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper jlong shuffleReaderHandle, jobject jniIn) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto reader = ctx->getShuffleReader(shuffleReaderHandle); + auto reader = ctx->objectStore()->retrieve(shuffleReaderHandle); std::shared_ptr in = std::make_shared(env, reader->getPool(), jniIn); auto outItr = reader->readStream(in); - return ctx->addResultIterator(outItr); + return ctx->objectStore()->save(outItr); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1027,9 +1041,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper_ jlong shuffleReaderHandle, jobject metrics) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto reader = ctx->getShuffleReader(shuffleReaderHandle); + auto reader = ctx->objectStore()->retrieve(shuffleReaderHandle); env->CallVoidMethod(metrics, shuffleReaderMetricsSetDecompressTime, reader->getDecompressTime()); env->CallVoidMethod(metrics, shuffleReaderMetricsSetIpcTime, reader->getIpcTime()); env->CallVoidMethod(metrics, shuffleReaderMetricsSetDeserializeTime, reader->getDeserializeTime()); @@ -1043,11 +1057,11 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ShuffleReaderJniWrapper_ jobject wrapper, jlong shuffleReaderHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto reader = ctx->getShuffleReader(shuffleReaderHandle); + auto reader = ctx->objectStore()->retrieve(shuffleReaderHandle); GLUTEN_THROW_NOT_OK(reader->close()); - ctx->releaseShuffleReader(shuffleReaderHandle); + ctx->objectStore()->release(shuffleReaderHandle); JNI_METHOD_END() } @@ -1059,21 +1073,21 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_datasource_DatasourceJniWrapper_na jlong memoryManagerHandle, jbyteArray options) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); ResourceHandle handle = kInvalidResourceHandle; if (cSchema == -1) { // Only inspect the schema and not write - handle = ctx->createDatasource(jStringToCString(env, filePath), memoryManager, nullptr); + handle = ctx->objectStore()->save(ctx->createDatasource(jStringToCString(env, filePath), memoryManager, nullptr)); } else { auto datasourceOptions = gluten::parseConfMap(env, options); auto& sparkConf = ctx->getConfMap(); datasourceOptions.insert(sparkConf.begin(), sparkConf.end()); auto schema = gluten::arrowGetOrThrow(arrow::ImportSchema(reinterpret_cast(cSchema))); - handle = ctx->createDatasource(jStringToCString(env, filePath), memoryManager, schema); - auto datasource = ctx->getDatasource(handle); + handle = ctx->objectStore()->save(ctx->createDatasource(jStringToCString(env, filePath), memoryManager, schema)); + auto datasource = ctx->objectStore()->retrieve(handle); datasource->init(datasourceOptions); } @@ -1087,9 +1101,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_datasource_DatasourceJniWrapper_ins jlong dsHandle, jlong cSchema) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto datasource = ctx->getDatasource(dsHandle); + auto datasource = ctx->objectStore()->retrieve(dsHandle); datasource->inspectSchema(reinterpret_cast(cSchema)); JNI_METHOD_END() } @@ -1099,11 +1113,11 @@ JNIEXPORT void JNICALL Java_io_glutenproject_datasource_DatasourceJniWrapper_clo jobject wrapper, jlong dsHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto datasource = ctx->getDatasource(dsHandle); + auto datasource = ctx->objectStore()->retrieve(dsHandle); datasource->close(); - ctx->releaseDatasource(dsHandle); + ctx->objectStore()->release(dsHandle); JNI_METHOD_END() } @@ -1113,8 +1127,8 @@ JNIEXPORT void JNICALL Java_io_glutenproject_datasource_DatasourceJniWrapper_wri jlong dsHandle, jobject jIter) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto datasource = ctx->getDatasource(dsHandle); + auto ctx = gluten::getRuntime(env, wrapper); + auto datasource = ctx->objectStore()->retrieve(dsHandle); auto iter = makeJniColumnarBatchIterator(env, jIter, ctx, nullptr); while (true) { auto batch = iter->next(); @@ -1130,13 +1144,13 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_datasource_DatasourceJniWrapper_splitBlockByPartitionAndBucket( // NOLINT JNIEnv* env, jobject wrapper, - jlong batchAddress, + jlong batchHandle, jintArray partitionColIndice, jboolean hasBucket, jlong memoryManagerId) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); - auto batch = ctx->getBatch(batchAddress); + auto ctx = gluten::getRuntime(env, wrapper); + auto batch = ctx->objectStore()->retrieve(batchHandle); int* pIndice = env->GetIntArrayElements(partitionColIndice, nullptr); int size = env->GetArrayLength(partitionColIndice); std::vector partitionColIndiceVec; @@ -1148,7 +1162,7 @@ Java_io_glutenproject_datasource_DatasourceJniWrapper_splitBlockByPartitionAndBu MemoryManager* memoryManager = reinterpret_cast(memoryManagerId); auto result = batch->getRowBytes(0); auto rowBytes = result.first; - auto batchHandler = ctx->select(memoryManager, batchAddress, partitionColIndiceVec); + auto newBatchHandle = ctx->objectStore()->save(ctx->select(memoryManager, batch, partitionColIndiceVec)); auto bytesSize = result.second; jbyteArray bytesArray = env->NewByteArray(bytesSize); @@ -1157,14 +1171,14 @@ Java_io_glutenproject_datasource_DatasourceJniWrapper_splitBlockByPartitionAndBu jlongArray batchArray = env->NewLongArray(1); long* cBatchArray = new long[1]; - cBatchArray[0] = batchHandler; + cBatchArray[0] = newBatchHandle; env->SetLongArrayRegion(batchArray, 0, 1, cBatchArray); delete[] cBatchArray; jobject block_stripes = env->NewObject( block_stripes_class, block_stripes_constructor, - batchAddress, + batchHandle, batchArray, nullptr, batch->numColumns(), @@ -1240,10 +1254,10 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_memory_nmm_NativeMemoryManager_cre auto name = jStringToCString(env, jnmmName); auto backendType = jStringToCString(env, jbackendType); - // TODO: move memory manager into ExecutionCtx then we can use more general ExecutionCtx. - auto executionCtx = gluten::ExecutionCtx::create(backendType); - auto manager = executionCtx->createMemoryManager(name, *allocator, std::move(listener)); - gluten::ExecutionCtx::release(executionCtx); + // TODO: move memory manager into Runtime then we can use more general Runtime. + auto runtime = gluten::Runtime::create(backendType); + auto manager = runtime->createMemoryManager(name, *allocator, std::move(listener)); + gluten::Runtime::release(runtime); return reinterpret_cast(manager); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1278,6 +1292,16 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_memory_nmm_NativeMemoryManager_shr JNI_METHOD_END(kInvalidResourceHandle) } +JNIEXPORT void JNICALL Java_io_glutenproject_memory_nmm_NativeMemoryManager_hold( // NOLINT + JNIEnv* env, + jclass, + jlong memoryManagerHandle) { + JNI_METHOD_START + auto memoryManager = jniCastOrThrow(memoryManagerHandle); + memoryManager->hold(); + JNI_METHOD_END() +} + JNIEXPORT void JNICALL Java_io_glutenproject_memory_nmm_NativeMemoryManager_release( // NOLINT JNIEnv* env, jclass, @@ -1294,7 +1318,7 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializ jlongArray handles, jlong memoryManagerHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); int32_t numBatches = env->GetArrayLength(handles); @@ -1303,7 +1327,7 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializ std::vector> batches; int64_t numRows = 0L; for (int32_t i = 0; i < numBatches; i++) { - auto batch = ctx->getBatch(batchHandles[i]); + auto batch = ctx->objectStore()->retrieve(batchHandles[i]); GLUTEN_DCHECK(batch != nullptr, "Cannot find the ColumnarBatch with handle " + std::to_string(batchHandles[i])); numRows += batch->numRows(); batches.emplace_back(batch); @@ -1311,7 +1335,7 @@ JNIEXPORT jobject JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializ env->ReleaseLongArrayElements(handles, batchHandles, JNI_ABORT); auto arrowPool = memoryManager->getArrowMemoryPool(); - auto serializer = ctx->createTempColumnarBatchSerializer(memoryManager, arrowPool, nullptr); + auto serializer = ctx->createColumnarBatchSerializer(memoryManager, arrowPool, nullptr); auto buffer = serializer->serializeColumnarBatches(batches); auto bufferArr = env->NewByteArray(buffer->size()); env->SetByteArrayRegion(bufferArr, 0, buffer->size(), reinterpret_cast(buffer->data())); @@ -1329,11 +1353,12 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializer jlong cSchema, jlong memoryManagerHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); auto memoryManager = jniCastOrThrow(memoryManagerHandle); auto arrowPool = memoryManager->getArrowMemoryPool(); - return ctx->createColumnarBatchSerializer(memoryManager, arrowPool, reinterpret_cast(cSchema)); + return ctx->objectStore()->save( + ctx->createColumnarBatchSerializer(memoryManager, arrowPool, reinterpret_cast(cSchema))); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1343,15 +1368,15 @@ JNIEXPORT jlong JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializer jlong serializerHandle, jbyteArray data) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - auto serializer = ctx->getColumnarBatchSerializer(serializerHandle); + auto serializer = ctx->objectStore()->retrieve(serializerHandle); GLUTEN_DCHECK(serializer != nullptr, "ColumnarBatchSerializer cannot be null"); int32_t size = env->GetArrayLength(data); jbyte* serialized = env->GetByteArrayElements(data, nullptr); auto batch = serializer->deserialize(reinterpret_cast(serialized), size); env->ReleaseByteArrayElements(data, serialized, JNI_ABORT); - return ctx->addBatch(batch); + return ctx->objectStore()->save(batch); JNI_METHOD_END(kInvalidResourceHandle) } @@ -1360,9 +1385,9 @@ JNIEXPORT void JNICALL Java_io_glutenproject_vectorized_ColumnarBatchSerializerJ jobject wrapper, jlong serializerHandle) { JNI_METHOD_START - auto ctx = gluten::getExecutionCtx(env, wrapper); + auto ctx = gluten::getRuntime(env, wrapper); - ctx->releaseColumnarBatchSerializer(serializerHandle); + ctx->objectStore()->release(serializerHandle); JNI_METHOD_END() } diff --git a/cpp/core/memory/ArrowMemoryPool.cc b/cpp/core/memory/ArrowMemoryPool.cc index 9d4bbd70cd10..28dd88fc1fab 100644 --- a/cpp/core/memory/ArrowMemoryPool.cc +++ b/cpp/core/memory/ArrowMemoryPool.cc @@ -16,6 +16,7 @@ */ #include "ArrowMemoryPool.h" +#include "arrow/type_fwd.h" #include "utils/exception.h" namespace gluten { diff --git a/cpp/core/memory/Evictable.h b/cpp/core/memory/Evictable.h new file mode 100644 index 000000000000..81b67d2d1c13 --- /dev/null +++ b/cpp/core/memory/Evictable.h @@ -0,0 +1,31 @@ +/* + * 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 gluten { + +class Evictable { + public: + virtual ~Evictable() = default; + + virtual arrow::Status evictFixedSize(int64_t size, int64_t* actual) = 0; +}; + +} // namespace gluten diff --git a/cpp/core/memory/MemoryManager.h b/cpp/core/memory/MemoryManager.h index 1a256596cc7c..5ec5213051ae 100644 --- a/cpp/core/memory/MemoryManager.h +++ b/cpp/core/memory/MemoryManager.h @@ -33,6 +33,11 @@ class MemoryManager { virtual const MemoryUsageStats collectMemoryUsageStats() const = 0; virtual const int64_t shrink(int64_t size) = 0; + + // Hold this memory manager. The underlying memory pools will be released as lately as this memory manager gets + // destroyed. Which means, a call to this function would make sure the memory blocks directly or indirectly managed + // by this manager, be guaranteed safe to access during the period that this manager is alive. + virtual void hold() = 0; }; } // namespace gluten diff --git a/cpp/core/resources/libhdfs.so b/cpp/core/resources/libhdfs.so deleted file mode 100755 index d9b2faaa486e..000000000000 Binary files a/cpp/core/resources/libhdfs.so and /dev/null differ diff --git a/cpp/core/resources/libhdfs3.so b/cpp/core/resources/libhdfs3.so deleted file mode 100755 index d9b2faaa486e..000000000000 Binary files a/cpp/core/resources/libhdfs3.so and /dev/null differ diff --git a/cpp/core/shuffle/FallbackRangePartitioner.h b/cpp/core/shuffle/FallbackRangePartitioner.h index 9594f880ceab..265915db925e 100644 --- a/cpp/core/shuffle/FallbackRangePartitioner.h +++ b/cpp/core/shuffle/FallbackRangePartitioner.h @@ -21,9 +21,9 @@ namespace gluten { -class FallbackRangePartitioner final : public ShuffleWriter::Partitioner { +class FallbackRangePartitioner final : public Partitioner { public: - FallbackRangePartitioner(int32_t numPartitions, bool hasPid) : Partitioner(numPartitions, hasPid) {} + FallbackRangePartitioner(int32_t numPartitions) : Partitioner(numPartitions, true) {} arrow::Status compute( const int32_t* pidArr, diff --git a/cpp/core/shuffle/HashPartitioner.h b/cpp/core/shuffle/HashPartitioner.h index e49b063ee7a3..0680f28b28ec 100644 --- a/cpp/core/shuffle/HashPartitioner.h +++ b/cpp/core/shuffle/HashPartitioner.h @@ -21,9 +21,9 @@ namespace gluten { -class HashPartitioner final : public ShuffleWriter::Partitioner { +class HashPartitioner final : public Partitioner { public: - HashPartitioner(int32_t numPartitions, bool hasPid) : Partitioner(numPartitions, hasPid) {} + HashPartitioner(int32_t numPartitions) : Partitioner(numPartitions, true) {} arrow::Status compute( const int32_t* pidArr, diff --git a/cpp/core/shuffle/LocalPartitionWriter.cc b/cpp/core/shuffle/LocalPartitionWriter.cc index e303027f6f2c..313be10653b8 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.cc +++ b/cpp/core/shuffle/LocalPartitionWriter.cc @@ -16,39 +16,173 @@ */ #include "shuffle/LocalPartitionWriter.h" +#include #include #include "shuffle/Utils.h" #include "utils/DebugOut.h" +#include "utils/StringUtil.h" #include "utils/Timer.h" namespace gluten { -std::string LocalPartitionWriterBase::nextSpilledFileDir() { +class LocalPartitionWriter::LocalEvictHandle : public EvictHandle { + public: + LocalEvictHandle( + uint32_t numPartitions, + const arrow::ipc::IpcWriteOptions& options, + const std::shared_ptr& spillInfo) + : numPartitions_(numPartitions), options_(options), spillInfo_(spillInfo) {} + + static std::shared_ptr create( + uint32_t numPartitions, + const arrow::ipc::IpcWriteOptions& options, + const std::shared_ptr& spillInfo, + bool flush); + + bool finished() { + return finished_; + }; + + virtual arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) = 0; + + protected: + uint32_t numPartitions_; + arrow::ipc::IpcWriteOptions options_; + std::shared_ptr spillInfo_; + + std::shared_ptr os_; + bool finished_{false}; +}; + +class CacheEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { + public: + CacheEvictHandle( + uint32_t numPartitions, + const arrow::ipc::IpcWriteOptions& options, + const std::shared_ptr& spillInfo) + : LocalPartitionWriter::LocalEvictHandle(numPartitions, options, spillInfo) { + partitionCachedPayload_.resize(numPartitions); + } + + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + partitionCachedPayload_[partitionId].push_back(std::move(payload)); + return arrow::Status::OK(); + } + + arrow::Status finish() override { + if (!finished_) { + ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); + int64_t start = 0; + for (uint32_t pid = 0; pid < numPartitions_; ++pid) { + if (!partitionCachedPayload_[pid].empty()) { + RETURN_NOT_OK(flushCachedPayloads(pid, os_.get())); + ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); + spillInfo_->partitionSpillInfos.push_back({pid, end - start}); + start = end; + } + } + partitionCachedPayload_.clear(); + ARROW_ASSIGN_OR_RAISE(auto written, os_->Tell()); + RETURN_NOT_OK(os_->Close()); + if (written > 0) { + spillInfo_->empty = false; + } + + finished_ = true; + } + return arrow::Status::OK(); + } + + arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) override { + if (partitionCachedPayload_[partitionId].empty()) { + return arrow::Status::OK(); + } + + int32_t metadataLength = 0; // unused + auto payloads = std::move(partitionCachedPayload_[partitionId]); + // Clear cached batches before creating the payloads, to avoid spilling this partition. + partitionCachedPayload_[partitionId].clear(); + for (auto& payload : payloads) { + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, os, &metadataLength)); + } + return arrow::Status::OK(); + } + + private: + std::vector>> partitionCachedPayload_; +}; + +class FlushOnSpillEvictHandle final : public LocalPartitionWriter::LocalEvictHandle { + public: + FlushOnSpillEvictHandle( + uint32_t numPartitions, + const arrow::ipc::IpcWriteOptions& options, + const std::shared_ptr& spillInfo) + : LocalPartitionWriter::LocalEvictHandle(numPartitions, options, spillInfo) {} + + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + if (!os_) { + ARROW_ASSIGN_OR_RAISE(os_, arrow::io::FileOutputStream::Open(spillInfo_->spilledFile, true)); + } + int32_t metadataLength = 0; // unused. + + ARROW_ASSIGN_OR_RAISE(auto start, os_->Tell()); + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, os_.get(), &metadataLength)); + ARROW_ASSIGN_OR_RAISE(auto end, os_->Tell()); + DEBUG_OUT << "Spilled partition " << partitionId << " file start: " << start << ", file end: " << end << std::endl; + spillInfo_->partitionSpillInfos.push_back({partitionId, end - start}); + return arrow::Status::OK(); + } + + arrow::Status finish() override { + if (!finished_) { + if (os_) { + RETURN_NOT_OK(os_->Close()); + spillInfo_->empty = false; + } + finished_ = true; + } + return arrow::Status::OK(); + } + + arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) override { + return arrow::Status::OK(); + } +}; + +std::shared_ptr LocalPartitionWriter::LocalEvictHandle::create( + uint32_t numPartitions, + const arrow::ipc::IpcWriteOptions& options, + const std::shared_ptr& spillInfo, + bool flush) { + if (flush) { + return std::make_shared(numPartitions, options, spillInfo); + } else { + return std::make_shared(numPartitions, options, spillInfo); + } +} + +std::string LocalPartitionWriter::nextSpilledFileDir() { auto spilledFileDir = getSpilledShuffleFileDir(configuredDirs_[dirSelection_], subDirSelection_[dirSelection_]); subDirSelection_[dirSelection_] = (subDirSelection_[dirSelection_] + 1) % shuffleWriter_->options().num_sub_dirs; dirSelection_ = (dirSelection_ + 1) % configuredDirs_.size(); return spilledFileDir; } -arrow::Status LocalPartitionWriterBase::setLocalDirs() { - ARROW_ASSIGN_OR_RAISE(configuredDirs_, getConfiguredLocalDirs()); +arrow::Status LocalPartitionWriter::setLocalDirs() { + configuredDirs_ = splitPaths(shuffleWriter_->options().local_dirs); + // Shuffle the configured local directories. This prevents each task from using the same directory for spilled files. + std::random_device rd; + std::default_random_engine engine(rd()); + std::shuffle(configuredDirs_.begin(), configuredDirs_.end(), engine); subDirSelection_.assign(configuredDirs_.size(), 0); - - // Both data_file and shuffle_index_file should be set through jni. - // For test purpose, Create a temporary subdirectory in the system temporary - // dir with prefix "columnar-shuffle" - if (shuffleWriter_->options().data_file.length() == 0) { - std::string dataFileTemp; - size_t id = std::hash{}(std::this_thread::get_id()) % configuredDirs_.size(); - ARROW_ASSIGN_OR_RAISE(shuffleWriter_->options().data_file, createTempShuffleFile(configuredDirs_[id])); - } return arrow::Status::OK(); } -arrow::Status LocalPartitionWriterBase::openDataFile() { +arrow::Status LocalPartitionWriter::openDataFile() { // open data file output stream std::shared_ptr fout; - ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(shuffleWriter_->options().data_file, true)); + ARROW_ASSIGN_OR_RAISE(fout, arrow::io::FileOutputStream::Open(shuffleWriter_->options().data_file)); if (shuffleWriter_->options().buffered_write) { // Output stream buffer is neither partition buffer memory nor ipc memory. ARROW_ASSIGN_OR_RAISE( @@ -59,52 +193,21 @@ arrow::Status LocalPartitionWriterBase::openDataFile() { return arrow::Status::OK(); } -arrow::Status LocalPartitionWriterBase::clearResource() { +arrow::Status LocalPartitionWriter::clearResource() { RETURN_NOT_OK(dataFileOs_->Close()); + // When buffered_write = true, dataFileOs_->Close doesn't release underlying buffer. + dataFileOs_.reset(); + spills_.clear(); return arrow::Status::OK(); } -arrow::Status PreferCachePartitionWriter::init() { - partitionCachedPayload_.resize(shuffleWriter_->numPartitions()); +arrow::Status LocalPartitionWriter::init() { fs_ = std::make_shared(); RETURN_NOT_OK(setLocalDirs()); return arrow::Status::OK(); } -arrow::Status PreferCachePartitionWriter::spill() { - int64_t evictTime = 0; - TIME_NANO_START(evictTime) - - ARROW_ASSIGN_OR_RAISE(auto spilledFile, createTempShuffleFile(nextSpilledFileDir())); - SpillInfo spillInfo{spilledFile}; - - // Spill all cached batches into one file, record their start and length. - ARROW_ASSIGN_OR_RAISE(auto spilledFileOs, arrow::io::FileOutputStream::Open(spilledFile, true)); - for (auto pid = 0; pid < shuffleWriter_->numPartitions(); ++pid) { - if (partitionCachedPayload_[pid].size() > 0) { - ARROW_ASSIGN_OR_RAISE(auto start, spilledFileOs->Tell()); - RETURN_NOT_OK(flushCachedPayloads(pid, spilledFileOs.get())); - ARROW_ASSIGN_OR_RAISE(auto end, spilledFileOs->Tell()); - spillInfo.partitionSpillInfos.push_back({pid, end - start}); - DEBUG_OUT << "Spilled partition " << pid << " file start: " << start << ", file end: " << end << std::endl; - } - } - RETURN_NOT_OK(spilledFileOs->Close()); - - TIME_NANO_END(evictTime) - shuffleWriter_->setTotalEvictTime(shuffleWriter_->totalEvictTime() + evictTime); - - if (!spillInfo.partitionSpillInfos.empty()) { - spills_.push_back(std::move(spillInfo)); - } else { - // No data spilled to this file. Delete the file and discard this SpillInfo. - RETURN_NOT_OK(fs_->DeleteFile(spilledFile)); - } - - return arrow::Status::OK(); -} - -arrow::Status PreferCachePartitionWriter::stop() { +arrow::Status LocalPartitionWriter::stop() { int64_t totalBytesEvicted = 0; int64_t totalBytesWritten = 0; auto numPartitions = shuffleWriter_->numPartitions(); @@ -116,67 +219,67 @@ arrow::Status PreferCachePartitionWriter::stop() { auto writeTimer = Timer(); writeTimer.start(); + int64_t endInFinalFile = 0; // Iterator over pid. for (auto pid = 0; pid < numPartitions; ++pid) { - bool firstWrite = true; // Record start offset. - ARROW_ASSIGN_OR_RAISE(auto startInFinalFile, dataFileOs_->Tell()); + auto startInFinalFile = endInFinalFile; // Iterator over all spilled files. - for (auto& spill : spills_) { + for (auto spill : spills_) { // Read if partition exists in the spilled file and write to the final file. - if (spill.mergePos < spill.partitionSpillInfos.size() && - spill.partitionSpillInfos[spill.mergePos].partitionId == pid) { // A hit. - if (!spill.inputStream) { + if (spill->mergePos < spill->partitionSpillInfos.size() && + spill->partitionSpillInfos[spill->mergePos].partitionId == pid) { // A hit. + if (!spill->inputStream) { // Open spilled file. ARROW_ASSIGN_OR_RAISE( - spill.inputStream, arrow::io::MemoryMappedFile::Open(spill.spilledFile, arrow::io::FileMode::READ)); + spill->inputStream, arrow::io::MemoryMappedFile::Open(spill->spilledFile, arrow::io::FileMode::READ)); // Add evict metrics. - ARROW_ASSIGN_OR_RAISE(auto spilledSize, spill.inputStream->GetSize()); + ARROW_ASSIGN_OR_RAISE(auto spilledSize, spill->inputStream->GetSize()); totalBytesEvicted += spilledSize; } - firstWrite = false; - auto spillInfo = spill.partitionSpillInfos[spill.mergePos]; - ARROW_ASSIGN_OR_RAISE(auto raw, spill.inputStream->Read(spillInfo.length)); + auto spillInfo = spill->partitionSpillInfos[spill->mergePos]; + ARROW_ASSIGN_OR_RAISE(auto raw, spill->inputStream->Read(spillInfo.length)); RETURN_NOT_OK(dataFileOs_->Write(raw)); // Goto next partition in this spillInfo. - spill.mergePos++; + spill->mergePos++; } } // Write cached batches. - if (!partitionCachedPayload_[pid].empty()) { - firstWrite = false; - RETURN_NOT_OK(flushCachedPayloads(pid, dataFileOs_.get())); + if (evictHandle_ && !evictHandle_->finished()) { + RETURN_NOT_OK(evictHandle_->flushCachedPayloads(pid, dataFileOs_.get())); } - // Write the last payload. + // Compress and write the last payload. + // Stop the timer to prevent counting the compression time into write time. writeTimer.stop(); ARROW_ASSIGN_OR_RAISE(auto lastPayload, shuffleWriter_->createPayloadFromBuffer(pid, false)); writeTimer.start(); if (lastPayload) { - firstWrite = false; int32_t metadataLength = 0; // unused - RETURN_NOT_OK(flushCachedPayload(dataFileOs_.get(), std::move(lastPayload), &metadataLength)); + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload( + *lastPayload, shuffleWriter_->options().ipc_write_options, dataFileOs_.get(), &metadataLength)); } - // Write EOS if any payload written. - if (shuffleWriter_->options().write_eos && !firstWrite) { - RETURN_NOT_OK(writeEos(dataFileOs_.get())); + ARROW_ASSIGN_OR_RAISE(endInFinalFile, dataFileOs_->Tell()); + if (endInFinalFile != startInFinalFile && shuffleWriter_->options().write_eos) { + // Write EOS if any payload written. + int64_t bytes; + RETURN_NOT_OK(writeEos(dataFileOs_.get(), &bytes)); + endInFinalFile += bytes; } - ARROW_ASSIGN_OR_RAISE(auto endInFinalFile, dataFileOs_->Tell()); shuffleWriter_->setPartitionLengths(pid, endInFinalFile - startInFinalFile); } - // Close spilled file streams and delete the file. - for (auto i = 0; i < spills_.size(); ++i) { + for (auto spill : spills_) { // Check if all spilled data are merged. - if (spills_[i].mergePos != spills_[i].partitionSpillInfos.size()) { - return arrow::Status::Invalid("Merging from spilled file NO." + std::to_string(i) + " is out of bound."); + if (spill->mergePos != spill->partitionSpillInfos.size()) { + return arrow::Status::Invalid("Merging from spilled file out of bound: " + spill->spilledFile); } - if (!spills_[i].inputStream) { - return arrow::Status::Invalid("Spilled file NO. " + std::to_string(i) + " has not been merged."); + // Close spilled file streams and delete the spilled file. + if (spill->inputStream) { + RETURN_NOT_OK(spill->inputStream->Close()); } - RETURN_NOT_OK(spills_[i].inputStream->Close()); - RETURN_NOT_OK(fs_->DeleteFile(spills_[i].spilledFile)); + RETURN_NOT_OK(fs_->DeleteFile(spill->spilledFile)); } ARROW_ASSIGN_OR_RAISE(totalBytesWritten, dataFileOs_->Tell()); @@ -193,15 +296,34 @@ arrow::Status PreferCachePartitionWriter::stop() { return arrow::Status::OK(); } -arrow::Status PreferCachePartitionWriter::clearResource() { - RETURN_NOT_OK(LocalPartitionWriterBase::clearResource()); - spills_.clear(); +arrow::Status LocalPartitionWriter::requestNextEvict(bool flush) { + RETURN_NOT_OK(finishEvict()); + ARROW_ASSIGN_OR_RAISE(auto spilledFile, createTempShuffleFile(nextSpilledFileDir())); + auto spillInfo = std::make_shared(spilledFile); + spills_.push_back(spillInfo); + evictHandle_ = LocalEvictHandle::create( + shuffleWriter_->numPartitions(), shuffleWriter_->options().ipc_write_options, spillInfo, flush); return arrow::Status::OK(); } -arrow::Status PreferCachePartitionWriter::processPayload( - uint32_t partitionId, - std::unique_ptr payload) { - partitionCachedPayload_[partitionId].push_back(std::move(payload)); + +EvictHandle* LocalPartitionWriter::getEvictHandle() { + if (evictHandle_ && !evictHandle_->finished()) { + return evictHandle_.get(); + } + return nullptr; +} + +arrow::Status LocalPartitionWriter::finishEvict() { + if (auto handle = getEvictHandle()) { + RETURN_NOT_OK(handle->finish()); + // The spilled file should not be empty. However, defensively + // discard the last SpillInfo to avoid iterating over invalid ones. + auto lastSpillInfo = spills_.back(); + if (lastSpillInfo->empty) { + RETURN_NOT_OK(fs_->DeleteFile(lastSpillInfo->spilledFile)); + spills_.pop_back(); + } + } return arrow::Status::OK(); } @@ -209,7 +331,7 @@ LocalPartitionWriterCreator::LocalPartitionWriterCreator() : PartitionWriterCrea arrow::Result> LocalPartitionWriterCreator::make( ShuffleWriter* shuffleWriter) { - auto partitionWriter = std::make_shared(shuffleWriter); + auto partitionWriter = std::make_shared(shuffleWriter); RETURN_NOT_OK(partitionWriter->init()); return partitionWriter; } diff --git a/cpp/core/shuffle/LocalPartitionWriter.h b/cpp/core/shuffle/LocalPartitionWriter.h index 801e70163576..27fa2c0c2c6f 100644 --- a/cpp/core/shuffle/LocalPartitionWriter.h +++ b/cpp/core/shuffle/LocalPartitionWriter.h @@ -28,35 +28,33 @@ namespace gluten { -class LocalPartitionWriterBase : public ShuffleWriter::PartitionWriter { - protected: - explicit LocalPartitionWriterBase(ShuffleWriter* shuffleWriter) : PartitionWriter(shuffleWriter) {} - - arrow::Status setLocalDirs(); - - std::string nextSpilledFileDir(); - - arrow::Status openDataFile(); +struct SpillInfo { + struct PartitionSpillInfo { + uint32_t partitionId{}; + int64_t length{}; // in Bytes + }; - virtual arrow::Status clearResource(); + bool empty{true}; + std::string spilledFile{}; + std::vector partitionSpillInfos{}; + std::shared_ptr inputStream{}; - // configured local dirs for spilled file - int32_t dirSelection_ = 0; - std::vector subDirSelection_; - std::vector configuredDirs_; + int32_t mergePos = 0; - std::shared_ptr dataFileOs_; + SpillInfo(std::string spilledFile) : spilledFile(spilledFile) {} }; -class PreferCachePartitionWriter : public LocalPartitionWriterBase { +class LocalPartitionWriter : public ShuffleWriter::PartitionWriter { public: - explicit PreferCachePartitionWriter(ShuffleWriter* shuffleWriter) : LocalPartitionWriterBase(shuffleWriter) {} + explicit LocalPartitionWriter(ShuffleWriter* shuffleWriter) : PartitionWriter(shuffleWriter) {} arrow::Status init() override; - arrow::Status processPayload(uint32_t partitionId, std::unique_ptr payload) override; + arrow::Status requestNextEvict(bool flush) override; + + EvictHandle* getEvictHandle() override; - arrow::Status spill() override; + arrow::Status finishEvict() override; /// The stop function performs several tasks: /// 1. Opens the final data file. @@ -79,48 +77,27 @@ class PreferCachePartitionWriter : public LocalPartitionWriterBase { /// it will shrink partition buffers to free more memory. arrow::Status stop() override; + class LocalEvictHandle; + private: - arrow::Status clearResource() override; - - arrow::Status flushCachedPayload( - arrow::io::OutputStream* os, - std::unique_ptr payload, - int32_t* metadataLength) { -#ifndef SKIPWRITE - RETURN_NOT_OK( - arrow::ipc::WriteIpcPayload(*payload, shuffleWriter_->options().ipc_write_options, os, metadataLength)); -#endif - return arrow::Status::OK(); - } - - arrow::Status flushCachedPayloads(uint32_t partitionId, arrow::io::OutputStream* os) { - int32_t metadataLength = 0; // unused - - auto payloads = std::move(partitionCachedPayload_[partitionId]); - // Clear cached batches before creating the payloads, to avoid spilling this partition. - partitionCachedPayload_[partitionId].clear(); - for (auto& payload : payloads) { - RETURN_NOT_OK(flushCachedPayload(os, std::move(payload), &metadataLength)); - } - return arrow::Status::OK(); - } + arrow::Status setLocalDirs(); - struct PartitionSpillInfo { - int32_t partitionId = -1; - int64_t length = 0; // in Bytes - }; + std::string nextSpilledFileDir(); - struct SpillInfo { - std::string spilledFile{}; - std::vector partitionSpillInfos{}; - std::shared_ptr inputStream{}; + arrow::Status openDataFile(); - int32_t mergePos = 0; - }; + arrow::Status clearResource(); - std::vector spills_; - std::vector>> partitionCachedPayload_; std::shared_ptr fs_{}; + std::shared_ptr evictHandle_; + std::vector> spills_; + + // configured local dirs for spilled file + int32_t dirSelection_ = 0; + std::vector subDirSelection_; + std::vector configuredDirs_; + + std::shared_ptr dataFileOs_; }; class LocalPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { diff --git a/cpp/core/shuffle/Options.cc b/cpp/core/shuffle/Options.cc new file mode 100644 index 000000000000..25fc22ea0b14 --- /dev/null +++ b/cpp/core/shuffle/Options.cc @@ -0,0 +1,26 @@ +/* + * 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 "shuffle/Options.h" + +gluten::ShuffleReaderOptions gluten::ShuffleReaderOptions::defaults() { + return {}; +} + +gluten::ShuffleWriterOptions gluten::ShuffleWriterOptions::defaults() { + return {}; +} diff --git a/cpp/core/shuffle/Options.h b/cpp/core/shuffle/Options.h new file mode 100644 index 000000000000..129b425dd3a6 --- /dev/null +++ b/cpp/core/shuffle/Options.h @@ -0,0 +1,71 @@ +/* + * 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 "shuffle/Partitioning.h" +#include "utils/Compression.h" + +namespace gluten { + +static constexpr int32_t kDefaultShuffleWriterBufferSize = 4096; +static constexpr int32_t kDefaultNumSubDirs = 64; +static constexpr int32_t kDefaultCompressionThreshold = 100; +static constexpr int32_t kDefaultBufferAlignment = 64; +static constexpr double kDefaultBufferReallocThreshold = 0.25; +static constexpr bool kEnableBufferedWrite = true; +static constexpr bool kWriteEos = true; + +enum PartitionWriterType { kLocal, kCeleborn }; + +struct ShuffleReaderOptions { + arrow::ipc::IpcReadOptions ipc_read_options = arrow::ipc::IpcReadOptions::Defaults(); + arrow::Compression::type compression_type = arrow::Compression::type::LZ4_FRAME; + CodecBackend codec_backend = CodecBackend::NONE; + + static ShuffleReaderOptions defaults(); +}; + +struct ShuffleWriterOptions { + int32_t buffer_size = kDefaultShuffleWriterBufferSize; + int32_t push_buffer_max_size = kDefaultShuffleWriterBufferSize; + int32_t num_sub_dirs = kDefaultNumSubDirs; + int32_t compression_threshold = kDefaultCompressionThreshold; + double buffer_realloc_threshold = kDefaultBufferReallocThreshold; + arrow::Compression::type compression_type = arrow::Compression::LZ4_FRAME; + CodecBackend codec_backend = CodecBackend::NONE; + CompressionMode compression_mode = CompressionMode::BUFFER; + bool buffered_write = kEnableBufferedWrite; + bool write_eos = kWriteEos; + + PartitionWriterType partition_writer_type = PartitionWriterType::kLocal; + Partitioning partitioning = Partitioning::kRoundRobin; + + int64_t thread_id = -1; + int64_t task_attempt_id = -1; + + arrow::ipc::IpcWriteOptions ipc_write_options = arrow::ipc::IpcWriteOptions::Defaults(); + + std::string data_file{}; + std::string local_dirs{}; + arrow::MemoryPool* memory_pool{}; + + static ShuffleWriterOptions defaults(); +}; + +} // namespace gluten diff --git a/cpp/core/shuffle/PartitionWriter.h b/cpp/core/shuffle/PartitionWriter.h index 5dbf5620d213..d87c20955c3c 100644 --- a/cpp/core/shuffle/PartitionWriter.h +++ b/cpp/core/shuffle/PartitionWriter.h @@ -17,10 +17,20 @@ #pragma once +#include "shuffle/Options.h" #include "shuffle/ShuffleWriter.h" namespace gluten { +class EvictHandle { + public: + virtual ~EvictHandle() = default; + + virtual arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) = 0; + + virtual arrow::Status finish() = 0; +}; + class ShuffleWriter::PartitionWriter { public: PartitionWriter(ShuffleWriter* shuffleWriter) : shuffleWriter_(shuffleWriter) {} @@ -28,11 +38,22 @@ class ShuffleWriter::PartitionWriter { virtual arrow::Status init() = 0; - virtual arrow::Status processPayload(uint32_t partitionId, std::unique_ptr payload) = 0; + virtual arrow::Status stop() = 0; + + /// Request next evict. The caller can use `requestNextEvict` to start a evict, and choose to call + /// `getEvictHandle()->evict()` immediately, or to call it latter somewhere else. + /// The caller can start new evict multiple times. Once it's called, the last `EvictHandle` + /// will be finished automatically. + /// \param flush Whether to flush the evicted data immediately. If it's false, + /// the data can be cached first. + virtual arrow::Status requestNextEvict(bool flush) = 0; - virtual arrow::Status spill() = 0; + /// Get the current managed EvictHandle. Returns nullptr if the current EvictHandle was finished, + /// or requestNextEvict has not been called. + /// \return + virtual EvictHandle* getEvictHandle() = 0; - virtual arrow::Status stop() = 0; + virtual arrow::Status finishEvict() = 0; ShuffleWriter* shuffleWriter_; }; diff --git a/cpp/core/shuffle/Partitioner.cc b/cpp/core/shuffle/Partitioner.cc index c73a83ecc09b..31afb6e190b0 100644 --- a/cpp/core/shuffle/Partitioner.cc +++ b/cpp/core/shuffle/Partitioner.cc @@ -19,28 +19,22 @@ #include "shuffle/FallbackRangePartitioner.h" #include "shuffle/HashPartitioner.h" #include "shuffle/RoundRobinPartitioner.h" -#include "shuffle/SinglePartPartitioner.h" +#include "shuffle/SinglePartitioner.h" namespace gluten { -arrow::Result> ShuffleWriter::Partitioner::make( - const std::string& name, - int32_t numPartitions) { - std::shared_ptr partitioner = nullptr; - if (name == "hash") { - partitioner = ShuffleWriter::Partitioner::create(numPartitions, true); - } else if (name == "rr") { - partitioner = ShuffleWriter::Partitioner::create(numPartitions, false); - } else if (name == "range") { - partitioner = ShuffleWriter::Partitioner::create(numPartitions, true); - } else if (name == "single") { - partitioner = ShuffleWriter::Partitioner::create(numPartitions, false); - } - - if (!partitioner) { - return arrow::Status::NotImplemented("Partitioning " + name + " not supported yet."); - } else { - return partitioner; +arrow::Result> Partitioner::make(Partitioning partitioning, int32_t numPartitions) { + switch (partitioning) { + case Partitioning::kHash: + return std::make_shared(numPartitions); + case Partitioning::kRoundRobin: + return std::make_shared(numPartitions); + case Partitioning::kSingle: + return std::make_shared(); + case Partitioning::kRange: + return std::make_shared(numPartitions); + default: + return arrow::Status::Invalid("Unsupported partitioning type: " + std::to_string(partitioning)); } } diff --git a/cpp/core/shuffle/Partitioner.h b/cpp/core/shuffle/Partitioner.h index fbb35465c285..c60a15cf45ce 100644 --- a/cpp/core/shuffle/Partitioner.h +++ b/cpp/core/shuffle/Partitioner.h @@ -17,22 +17,18 @@ #pragma once -#include "shuffle/ShuffleWriter.h" +#include +#include +#include +#include "shuffle/Partitioning.h" namespace gluten { -class ShuffleWriter::Partitioner { +class Partitioner { public: - template - static std::shared_ptr create(int32_t numPartitions, bool hasPid) { - return std::make_shared(numPartitions, hasPid); - } - - static arrow::Result> make( - const std::string& name, - int32_t numPartitions); + static arrow::Result> make(Partitioning partitioning, int32_t numPartitions); - // whether the first column is partition key + // Whether the first column is partition key. bool hasPid() const { return hasPid_; } @@ -46,11 +42,11 @@ class ShuffleWriter::Partitioner { protected: Partitioner(int32_t numPartitions, bool hasPid) : numPartitions_(numPartitions), hasPid_(hasPid) {} + Partitioner() : numPartitions_(1), hasPid_(false) {} + virtual ~Partitioner() = default; int32_t numPartitions_; - - // if the first column is partition key bool hasPid_; }; diff --git a/cpp/core/shuffle/Partitioning.cc b/cpp/core/shuffle/Partitioning.cc new file mode 100644 index 000000000000..dfe848d63046 --- /dev/null +++ b/cpp/core/shuffle/Partitioning.cc @@ -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. + */ + +#include "shuffle/Partitioning.h" +#include "utils/exception.h" + +namespace { +static const std::string kSinglePartitioningName = "single"; +static const std::string kRoundRobinPartitioningName = "rr"; +static const std::string kHashPartitioningName = "hash"; +static const std::string kRangePartitioningName = "range"; +} // namespace + +namespace gluten { +Partitioning toPartitioning(std::string name) { + if (name == kSinglePartitioningName) { + return Partitioning::kSingle; + } + if (name == kRoundRobinPartitioningName) { + return Partitioning::kRoundRobin; + } + if (name == kHashPartitioningName) { + return Partitioning::kHash; + } + if (name == kRangePartitioningName) { + return Partitioning::kRange; + } + throw GlutenException("Invalid partition name: " + name); +} + +} // namespace gluten diff --git a/cpp/core/shuffle/Partitioning.h b/cpp/core/shuffle/Partitioning.h new file mode 100644 index 000000000000..1d65e9d6b993 --- /dev/null +++ b/cpp/core/shuffle/Partitioning.h @@ -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. + */ + +#pragma once + +#include + +namespace gluten { +enum Partitioning { kSingle, kRoundRobin, kHash, kRange }; + +Partitioning toPartitioning(std::string name); + +} // namespace gluten diff --git a/cpp/core/shuffle/RoundRobinPartitioner.h b/cpp/core/shuffle/RoundRobinPartitioner.h index 20eb4bab447e..8ea15e5afc7c 100644 --- a/cpp/core/shuffle/RoundRobinPartitioner.h +++ b/cpp/core/shuffle/RoundRobinPartitioner.h @@ -21,9 +21,9 @@ namespace gluten { -class RoundRobinPartitioner final : public ShuffleWriter::Partitioner { +class RoundRobinPartitioner final : public Partitioner { public: - RoundRobinPartitioner(int32_t numPartitions, bool hasPid) : Partitioner(numPartitions, hasPid) {} + RoundRobinPartitioner(int32_t numPartitions) : Partitioner(numPartitions, false) {} arrow::Status compute( const int32_t* pidArr, diff --git a/cpp/core/shuffle/ShuffleReader.cc b/cpp/core/shuffle/ShuffleReader.cc index 91b1e442d3bf..b1b278f581f3 100644 --- a/cpp/core/shuffle/ShuffleReader.cc +++ b/cpp/core/shuffle/ShuffleReader.cc @@ -30,7 +30,7 @@ using namespace gluten; class ShuffleReaderOutStream : public ColumnarBatchIterator { public: ShuffleReaderOutStream( - const ReaderOptions& options, + const ShuffleReaderOptions& options, const std::shared_ptr& schema, const std::shared_ptr& in, const std::function ipcTimeAccumulator) @@ -65,7 +65,7 @@ class ShuffleReaderOutStream : public ColumnarBatchIterator { } private: - ReaderOptions options_; + ShuffleReaderOptions options_; std::shared_ptr in_; std::function ipcTimeAccumulator_; std::shared_ptr writeSchema_; @@ -74,11 +74,10 @@ class ShuffleReaderOutStream : public ColumnarBatchIterator { namespace gluten { -ReaderOptions ReaderOptions::defaults() { - return {}; -} - -ShuffleReader::ShuffleReader(std::shared_ptr schema, ReaderOptions options, arrow::MemoryPool* pool) +ShuffleReader::ShuffleReader( + std::shared_ptr schema, + ShuffleReaderOptions options, + arrow::MemoryPool* pool) : pool_(pool), options_(std::move(options)), schema_(schema) {} std::shared_ptr ShuffleReader::readStream(std::shared_ptr in) { diff --git a/cpp/core/shuffle/ShuffleReader.h b/cpp/core/shuffle/ShuffleReader.h index 409aa63ab668..676211afb4ff 100644 --- a/cpp/core/shuffle/ShuffleReader.h +++ b/cpp/core/shuffle/ShuffleReader.h @@ -22,23 +22,15 @@ #include #include +#include "Options.h" #include "compute/ResultIterator.h" -#include "utils/compression.h" +#include "utils/Compression.h" namespace gluten { -struct ReaderOptions { - arrow::ipc::IpcReadOptions ipc_read_options = arrow::ipc::IpcReadOptions::Defaults(); - arrow::Compression::type compression_type = arrow::Compression::type::LZ4_FRAME; - CodecBackend codec_backend = CodecBackend::NONE; - CompressionMode compression_mode = CompressionMode::BUFFER; - - static ReaderOptions defaults(); -}; - class ShuffleReader { public: - explicit ShuffleReader(std::shared_ptr schema, ReaderOptions options, arrow::MemoryPool* pool); + explicit ShuffleReader(std::shared_ptr schema, ShuffleReaderOptions options, arrow::MemoryPool* pool); virtual ~ShuffleReader() = default; @@ -67,7 +59,7 @@ class ShuffleReader { int64_t ipcTime_ = 0; int64_t deserializeTime_ = 0; - ReaderOptions options_; + ShuffleReaderOptions options_; private: std::shared_ptr schema_; diff --git a/cpp/core/shuffle/ShuffleSchema.h b/cpp/core/shuffle/ShuffleSchema.h index 5d442e5656f2..dd906fdc6cce 100644 --- a/cpp/core/shuffle/ShuffleSchema.h +++ b/cpp/core/shuffle/ShuffleSchema.h @@ -29,7 +29,7 @@ inline std::shared_ptr toWriteSchema(arrow::Schema& schema) { case arrow::BinaryType::type_id: case arrow::StringType::type_id: { fields.emplace_back(std::make_shared("nullBuffer" + std::to_string(i), arrow::large_utf8())); - fields.emplace_back(std::make_shared("offsetBuffer" + std::to_string(i), arrow::large_utf8())); + fields.emplace_back(std::make_shared("lengthBuffer" + std::to_string(i), arrow::large_utf8())); fields.emplace_back(std::make_shared("valueBuffer" + std::to_string(i), arrow::large_utf8())); } break; case arrow::StructType::type_id: diff --git a/cpp/core/shuffle/ShuffleWriter.cc b/cpp/core/shuffle/ShuffleWriter.cc index c199f41d2b1d..e58869fdf548 100644 --- a/cpp/core/shuffle/ShuffleWriter.cc +++ b/cpp/core/shuffle/ShuffleWriter.cc @@ -31,10 +31,6 @@ namespace gluten { #define SPLIT_BUFFER_SIZE 16 * 1024 * 1024 #endif -ShuffleWriterOptions ShuffleWriterOptions::defaults() { - return {}; -} - std::shared_ptr ShuffleWriter::writeSchema() { if (writeSchema_ != nullptr) { return writeSchema_; diff --git a/cpp/core/shuffle/ShuffleWriter.h b/cpp/core/shuffle/ShuffleWriter.h index a87dd861b03e..df075a9984fe 100644 --- a/cpp/core/shuffle/ShuffleWriter.h +++ b/cpp/core/shuffle/ShuffleWriter.h @@ -23,47 +23,14 @@ #include "memory/ArrowMemoryPool.h" #include "memory/ColumnarBatch.h" -#include "utils/compression.h" +#include "memory/Evictable.h" +#include "shuffle/Options.h" +#include "shuffle/Partitioner.h" +#include "shuffle/Partitioning.h" +#include "utils/Compression.h" namespace gluten { -namespace { -static constexpr int32_t kDefaultShuffleWriterBufferSize = 4096; -static constexpr int32_t kDefaultNumSubDirs = 64; -static constexpr int32_t kDefaultBufferCompressThreshold = 1024; -static constexpr int32_t kDefaultBufferAlignment = 64; -static constexpr double kDefaultBufferReallocThreshold = 0.25; -} // namespace - -enum PartitionWriterType { kLocal, kCeleborn }; - -struct ShuffleWriterOptions { - int32_t buffer_size = kDefaultShuffleWriterBufferSize; - int32_t push_buffer_max_size = kDefaultShuffleWriterBufferSize; - int32_t num_sub_dirs = kDefaultNumSubDirs; - int32_t buffer_compress_threshold = kDefaultBufferCompressThreshold; - double buffer_realloc_threshold = kDefaultBufferReallocThreshold; - arrow::Compression::type compression_type = arrow::Compression::LZ4_FRAME; - CodecBackend codec_backend = CodecBackend::NONE; - CompressionMode compression_mode = CompressionMode::BUFFER; - bool buffered_write = false; - bool write_eos = true; - - std::string data_file; - PartitionWriterType partition_writer_type = kLocal; - - int64_t thread_id = -1; - int64_t task_attempt_id = -1; - - arrow::MemoryPool* memory_pool; - - arrow::ipc::IpcWriteOptions ipc_write_options = arrow::ipc::IpcWriteOptions::Defaults(); - - std::string partitioning_name; - - static ShuffleWriterOptions defaults(); -}; - class ShuffleMemoryPool : public arrow::MemoryPool { public: ShuffleMemoryPool(arrow::MemoryPool* pool) : pool_(pool) {} @@ -118,13 +85,9 @@ class ShuffleMemoryPool : public arrow::MemoryPool { uint64_t bytesAllocated_ = 0; }; -class ShuffleWriter { +class ShuffleWriter : public Evictable { public: static constexpr int64_t kMinMemLimit = 128LL * 1024 * 1024; - /** - * Evict fixed size of partition data from memory - */ - virtual arrow::Status evictFixedSize(int64_t size, int64_t* actual) = 0; virtual arrow::Status split(std::shared_ptr cb, int64_t memLimit) = 0; @@ -136,10 +99,6 @@ class ShuffleWriter { virtual arrow::Status stop() = 0; - virtual std::shared_ptr writeSchema(); - - virtual std::shared_ptr compressWriteSchema(); - virtual std::shared_ptr& schema() { return schema_; } @@ -212,8 +171,6 @@ class ShuffleWriter { class PartitionWriter; - class Partitioner; - class PartitionWriterCreator; protected: @@ -229,6 +186,10 @@ class ShuffleWriter { virtual ~ShuffleWriter() = default; + std::shared_ptr writeSchema(); + + std::shared_ptr compressWriteSchema(); + int32_t numPartitions_; std::shared_ptr partitionWriterCreator_; @@ -243,7 +204,6 @@ class ShuffleWriter { int64_t totalWriteTime_ = 0; int64_t totalEvictTime_ = 0; int64_t totalCompressTime_ = 0; - int64_t peakMemoryAllocated_ = 0; std::vector partitionLengths_; std::vector rawPartitionLengths_; // Uncompressed size. diff --git a/cpp/core/shuffle/SinglePartPartitioner.cc b/cpp/core/shuffle/SinglePartitioner.cc similarity index 91% rename from cpp/core/shuffle/SinglePartPartitioner.cc rename to cpp/core/shuffle/SinglePartitioner.cc index 3d75e97523e5..97c590380c11 100644 --- a/cpp/core/shuffle/SinglePartPartitioner.cc +++ b/cpp/core/shuffle/SinglePartitioner.cc @@ -15,11 +15,11 @@ * limitations under the License. */ -#include "shuffle/SinglePartPartitioner.h" +#include "shuffle/SinglePartitioner.h" namespace gluten { -arrow::Status gluten::SinglePartPartitioner::compute( +arrow::Status gluten::SinglePartitioner::compute( const int32_t* pidArr, const int64_t numRows, std::vector& row2partition, diff --git a/cpp/core/shuffle/SinglePartPartitioner.h b/cpp/core/shuffle/SinglePartitioner.h similarity index 86% rename from cpp/core/shuffle/SinglePartPartitioner.h rename to cpp/core/shuffle/SinglePartitioner.h index 4b3eb3374a71..3b386e45d374 100644 --- a/cpp/core/shuffle/SinglePartPartitioner.h +++ b/cpp/core/shuffle/SinglePartitioner.h @@ -20,9 +20,9 @@ #include "shuffle/Partitioner.h" namespace gluten { -class SinglePartPartitioner final : public ShuffleWriter::Partitioner { +class SinglePartitioner final : public Partitioner { public: - SinglePartPartitioner(int32_t numPartitions, bool hasPid) : Partitioner(numPartitions, hasPid) {} + SinglePartitioner() : Partitioner(1, false) {} arrow::Status compute( const int32_t* pidArr, @@ -30,5 +30,4 @@ class SinglePartPartitioner final : public ShuffleWriter::Partitioner { std::vector& row2partition, std::vector& partition2RowCount) override; }; - } // namespace gluten diff --git a/cpp/core/shuffle/Utils.cc b/cpp/core/shuffle/Utils.cc index dd7406ce6997..9255d679801c 100644 --- a/cpp/core/shuffle/Utils.cc +++ b/cpp/core/shuffle/Utils.cc @@ -16,6 +16,8 @@ */ #include "shuffle/Utils.h" +#include "Options.h" +#include "utils/StringUtil.h" #include #include @@ -39,31 +41,6 @@ std::string gluten::getSpilledShuffleFileDir(const std::string& configuredDir, i return dir; } -arrow::Result> gluten::getConfiguredLocalDirs() { - auto joinedDirsC = std::getenv(kGlutenSparkLocalDirs.c_str()); - if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { - auto joinedDirs = std::string(joinedDirsC); - std::string delimiter = ","; - - size_t pos; - std::vector res; - while ((pos = joinedDirs.find(delimiter)) != std::string::npos) { - auto dir = joinedDirs.substr(0, pos); - if (dir.length() > 0) { - res.push_back(std::move(dir)); - } - joinedDirs.erase(0, pos + delimiter.length()); - } - if (joinedDirs.length() > 0) { - res.push_back(std::move(joinedDirs)); - } - return res; - } else { - ARROW_ASSIGN_OR_RAISE(auto arrow_tmp_dir, arrow::internal::TemporaryDir::Make("columnar-shuffle-")); - return std::vector{arrow_tmp_dir->path().ToString()}; - } -} - arrow::Result gluten::createTempShuffleFile(const std::string& dir) { if (dir.length() == 0) { return arrow::Status::Invalid("Failed to create spilled file, got empty path."); @@ -139,11 +116,7 @@ arrow::Result>> gluten::toShuffleWr } int64_t gluten::getBufferSizes(const std::shared_ptr& array) { - const auto& buffers = array->data()->buffers; - return std::accumulate( - std::cbegin(buffers), std::cend(buffers), 0LL, [](int64_t sum, const std::shared_ptr& buf) { - return buf == nullptr ? sum : sum + buf->size(); - }); + return gluten::getBufferSizes(array->data()->buffers); } int64_t gluten::getBufferSizes(const std::vector>& buffers) { @@ -153,11 +126,13 @@ int64_t gluten::getBufferSizes(const std::vector> }); } -arrow::Status gluten::writeEos(arrow::io::OutputStream* os) { +arrow::Status gluten::writeEos(arrow::io::OutputStream* os, int64_t* bytes) { // write EOS - constexpr int32_t kIpcContinuationToken = -1; - constexpr int32_t kZeroLength = 0; - RETURN_NOT_OK(os->Write(&kIpcContinuationToken, sizeof(int32_t))); - RETURN_NOT_OK(os->Write(&kZeroLength, sizeof(int32_t))); + static constexpr int32_t kIpcContinuationToken = -1; + static constexpr int32_t kZeroLength = 0; + static const int64_t kSizeOfEos = sizeof(kIpcContinuationToken) + sizeof(kZeroLength); + RETURN_NOT_OK(os->Write(&kIpcContinuationToken, sizeof(kIpcContinuationToken))); + RETURN_NOT_OK(os->Write(&kZeroLength, sizeof(kZeroLength))); + *bytes = kSizeOfEos; return arrow::Status::OK(); } diff --git a/cpp/core/shuffle/Utils.h b/cpp/core/shuffle/Utils.h index 0cba0f996aaf..9c104d759828 100644 --- a/cpp/core/shuffle/Utils.h +++ b/cpp/core/shuffle/Utils.h @@ -30,25 +30,10 @@ namespace gluten { const std::string kGlutenSparkLocalDirs = "GLUTEN_SPARK_LOCAL_DIRS"; -#define EVAL_START(name, thread_id) \ - // auto eval_start = std::chrono::duration_cast( \ - std::chrono::system_clock::now().time_since_epoch()) \ - .count(); - -#define EVAL_END(name, thread_id, task_attempt_id) \ - // std::cout << "xgbtck " << name << " " << eval_start << " " \ - << std::chrono::duration_cast( \ - std::chrono::system_clock::now().time_since_epoch()) \ - .count() - \ - eval_start \ - << " " << thread_id << " " << task_attempt_id << std::endl; - std::string generateUuid(); std::string getSpilledShuffleFileDir(const std::string& configuredDir, int32_t subDirId); -arrow::Result> getConfiguredLocalDirs(); - arrow::Result createTempShuffleFile(const std::string& dir); arrow::Result>> toShuffleWriterTypeId( @@ -58,6 +43,6 @@ int64_t getBufferSizes(const std::shared_ptr& array); int64_t getBufferSizes(const std::vector>& buffers); -arrow::Status writeEos(arrow::io::OutputStream* os); +arrow::Status writeEos(arrow::io::OutputStream* os, int64_t* bytes); } // namespace gluten diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc index 854700d48696..e3ccbeec3161 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.cc +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.cc @@ -19,13 +19,48 @@ namespace gluten { -arrow::Status CelebornPartitionWriter::init() { - return arrow::Status::OK(); -} +class CelebornEvictHandle final : public EvictHandle { + public: + CelebornEvictHandle( + int64_t bufferSize, + const arrow::ipc::IpcWriteOptions& options, + arrow::MemoryPool* pool, + RssClient* client, + std::vector& bytesEvicted) + : bufferSize_(bufferSize), options_(options), pool_(pool), client_(client), bytesEvicted_(bytesEvicted) {} + + arrow::Status evict(uint32_t partitionId, std::unique_ptr payload) override { + // Copy payload to arrow buffered os. + ARROW_ASSIGN_OR_RAISE(auto celebornBufferOs, arrow::io::BufferOutputStream::Create(bufferSize_, pool_)); + int32_t metadataLength = 0; // unused + RETURN_NOT_OK(arrow::ipc::WriteIpcPayload(*payload, options_, celebornBufferOs.get(), &metadataLength)); + payload = nullptr; // Invalidate payload immediately. + + // Push. + ARROW_ASSIGN_OR_RAISE(auto buffer, celebornBufferOs->Finish()); + bytesEvicted_[partitionId] += client_->pushPartitionData( + partitionId, reinterpret_cast(const_cast(buffer->data())), buffer->size()); + return arrow::Status::OK(); + } + + arrow::Status finish() override { + return arrow::Status::OK(); + } + + private: + int64_t bufferSize_; + arrow::ipc::IpcWriteOptions options_; + arrow::MemoryPool* pool_; + RssClient* client_; -arrow::Status CelebornPartitionWriter::pushPartition(int32_t partitionId, char* data, int64_t size) { - int32_t celebornBytesSize = celebornClient_->pushPartitionData(partitionId, data, size); - shuffleWriter_->setPartitionLengths(partitionId, shuffleWriter_->partitionLengths()[partitionId] + celebornBytesSize); + std::vector& bytesEvicted_; +}; + +arrow::Status CelebornPartitionWriter::init() { + const auto& options = shuffleWriter_->options(); + bytesEvicted_.resize(shuffleWriter_->numPartitions(), 0); + evictHandle_ = std::make_shared( + options.buffer_size, options.ipc_write_options, options.memory_pool, celebornClient_.get(), bytesEvicted_); return arrow::Status::OK(); } @@ -34,46 +69,25 @@ arrow::Status CelebornPartitionWriter::stop() { for (auto pid = 0; pid < shuffleWriter_->numPartitions(); ++pid) { ARROW_ASSIGN_OR_RAISE(auto payload, shuffleWriter_->createPayloadFromBuffer(pid, false)); if (payload) { - RETURN_NOT_OK(processPayload(pid, std::move(payload))); + RETURN_NOT_OK(evictHandle_->evict(pid, std::move(payload))); } - shuffleWriter_->setTotalBytesWritten(shuffleWriter_->totalBytesWritten() + shuffleWriter_->partitionLengths()[pid]); + shuffleWriter_->setPartitionLengths(pid, bytesEvicted_[pid]); + shuffleWriter_->setTotalBytesWritten(shuffleWriter_->totalBytesWritten() + bytesEvicted_[pid]); } celebornClient_->stop(); return arrow::Status::OK(); } -arrow::Status CelebornPartitionWriter::processPayload( - uint32_t partitionId, - std::unique_ptr payload) { - // Copy payload to arrow buffered os. - int64_t writeTime = 0; - TIME_NANO_START(writeTime) - ARROW_ASSIGN_OR_RAISE( - auto celebornBufferOs, - arrow::io::BufferOutputStream::Create( - shuffleWriter_->options().buffer_size, shuffleWriter_->options().memory_pool)); - int32_t metadataLength = 0; // unused -#ifndef SKIPWRITE - RETURN_NOT_OK(arrow::ipc::WriteIpcPayload( - *payload, shuffleWriter_->options().ipc_write_options, celebornBufferOs.get(), &metadataLength)); -#endif - payload = nullptr; // Invalidate payload immediately. - TIME_NANO_END(writeTime) - shuffleWriter_->setTotalWriteTime(shuffleWriter_->totalWriteTime() + writeTime); - - // Push. - int64_t evictTime = 0; - ARROW_ASSIGN_OR_RAISE(auto buffer, celebornBufferOs->Finish()); - TIME_NANO_OR_RAISE( - evictTime, - pushPartition(partitionId, reinterpret_cast(const_cast(buffer->data())), buffer->size())); - shuffleWriter_->setTotalEvictTime(shuffleWriter_->totalEvictTime() + evictTime); +arrow::Status CelebornPartitionWriter::requestNextEvict(bool flush) { return arrow::Status::OK(); } -arrow::Status CelebornPartitionWriter::spill() { - // No-op because there's no cached data to spill. - return arrow::Status::OK(); +EvictHandle* CelebornPartitionWriter::getEvictHandle() { + return evictHandle_.get(); +} + +arrow::Status CelebornPartitionWriter::finishEvict() { + return evictHandle_->finish(); } CelebornPartitionWriterCreator::CelebornPartitionWriterCreator(std::shared_ptr client) diff --git a/cpp/core/shuffle/rss/CelebornPartitionWriter.h b/cpp/core/shuffle/rss/CelebornPartitionWriter.h index cc9a9507e869..3ca73d6d7af4 100644 --- a/cpp/core/shuffle/rss/CelebornPartitionWriter.h +++ b/cpp/core/shuffle/rss/CelebornPartitionWriter.h @@ -27,24 +27,29 @@ namespace gluten { -class CelebornPartitionWriter : public RemotePartitionWriter { +class CelebornPartitionWriter final : public RemotePartitionWriter { public: CelebornPartitionWriter(ShuffleWriter* shuffleWriter, std::shared_ptr celebornClient) : RemotePartitionWriter(shuffleWriter) { celebornClient_ = celebornClient; } - arrow::Status init() override; + arrow::Status requestNextEvict(bool flush /*unused*/) override; - arrow::Status processPayload(uint32_t partitionId, std::unique_ptr payload) override; + EvictHandle* getEvictHandle() override; - arrow::Status spill() override; + arrow::Status finishEvict() override; - arrow::Status stop() override; + arrow::Status init() override; - arrow::Status pushPartition(int32_t partitionId, char* data, int64_t size); + arrow::Status stop() override; + private: std::shared_ptr celebornClient_; + + std::shared_ptr evictHandle_; + + std::vector bytesEvicted_; }; class CelebornPartitionWriterCreator : public ShuffleWriter::PartitionWriterCreator { diff --git a/cpp/core/tests/RoundRobinPartitionerTest.cc b/cpp/core/tests/RoundRobinPartitionerTest.cc index 2cc1ed5d5198..5fb3e00feb19 100644 --- a/cpp/core/tests/RoundRobinPartitionerTest.cc +++ b/cpp/core/tests/RoundRobinPartitionerTest.cc @@ -16,13 +16,13 @@ */ #include "shuffle/RoundRobinPartitioner.h" #include -#include #include + namespace gluten { class RoundRobinPartitionerTest : public ::testing::Test { protected: void prepareData(int numPart) { - partitioner_ = ShuffleWriter::Partitioner::create(numPart, false); + partitioner_ = std::make_shared(numPart); row2Partition_.clear(); partition2RowCount_.clear(); partition2RowCount_.resize(numPart); diff --git a/cpp/core/utils/Compression.cc b/cpp/core/utils/Compression.cc index c84c3fc6875a..6469c95c197b 100644 --- a/cpp/core/utils/Compression.cc +++ b/cpp/core/utils/Compression.cc @@ -15,7 +15,7 @@ * limitations under the License. */ -#include "utils/compression.h" +#include "utils/Compression.h" #include "exception.h" diff --git a/cpp/core/utils/compression.h b/cpp/core/utils/Compression.h similarity index 100% rename from cpp/core/utils/compression.h rename to cpp/core/utils/Compression.h diff --git a/cpp/core/utils/ObjectStore.cc b/cpp/core/utils/ObjectStore.cc new file mode 100644 index 000000000000..4a8e7c7ea2f4 --- /dev/null +++ b/cpp/core/utils/ObjectStore.cc @@ -0,0 +1,44 @@ +/* + * 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 "ObjectStore.h" +#include + +gluten::ObjectStore::~ObjectStore() { + // destructing in reversed order (the last added object destructed first) + const std::lock_guard lock(mtx_); + for (auto itr = aliveObjectHandles_.rbegin(); itr != aliveObjectHandles_.rend(); itr++) { + ResourceHandle handle = *itr; + if (store_.lookup(handle) == nullptr) { + std::cerr << "Fatal: resource handle " + std::to_string(handle) + " not found in store." << std::endl; + } + store_.erase(handle); + } +} + +gluten::ResourceHandle gluten::ObjectStore::save(std::shared_ptr obj) { + const std::lock_guard lock(mtx_); + ResourceHandle handle = store_.insert(std::move(obj)); + aliveObjectHandles_.insert(handle); + return handle; +} + +void gluten::ObjectStore::release(gluten::ResourceHandle handle) { + const std::lock_guard lock(mtx_); + store_.erase(handle); + aliveObjectHandles_.erase(handle); +} diff --git a/cpp/core/utils/ObjectStore.h b/cpp/core/utils/ObjectStore.h new file mode 100644 index 000000000000..6bfecf599b0d --- /dev/null +++ b/cpp/core/utils/ObjectStore.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 "utils/ResourceMap.h" +#include "utils/exception.h" + +namespace gluten { + +// A store for caching shared-ptrs and enlarging lifecycles of the ptrs to match lifecycle of the store itself by +// default, and also serving release calls to release a ptr in advance. This is typically used in JNI scenario to bind +// a shared-ptr's lifecycle to a Java-side object or some kind of resource manager. +class ObjectStore { + public: + static std::unique_ptr create() { + return std::unique_ptr(new ObjectStore()); + } + + virtual ~ObjectStore(); + + ResourceHandle save(std::shared_ptr obj); + + template + std::shared_ptr retrieve(ResourceHandle handle) { + const std::lock_guard lock(mtx_); + std::shared_ptr object = store_.lookup(handle); + // Programming carefully. This will lead to ub if wrong typename T was passed in. + auto casted = std::static_pointer_cast(object); + return casted; + } + + void release(ResourceHandle handle); + + private: + ObjectStore(){}; + ResourceMap> store_; + std::set aliveObjectHandles_; + std::mutex mtx_; +}; +} // namespace gluten diff --git a/cpp/velox/utils/ResourceMap.h b/cpp/core/utils/ResourceMap.h similarity index 84% rename from cpp/velox/utils/ResourceMap.h rename to cpp/core/utils/ResourceMap.h index 5bc6ebfb2ce2..c47fbd9ecb39 100644 --- a/cpp/velox/utils/ResourceMap.h +++ b/cpp/core/utils/ResourceMap.h @@ -18,10 +18,13 @@ #pragma once #include -#include "folly/container/F14Map.h" +#include namespace gluten { +using ResourceHandle = int64_t; +constexpr static ResourceHandle kInvalidResourceHandle = -1; + /** * An utility class that map resource handle to its shared pointers. * Not thread-safe. @@ -33,19 +36,16 @@ class ResourceMap { ResourceMap() : resourceId_(kInitResourceId) {} ResourceHandle insert(TResource holder) { - const std::lock_guard lock(mtx_); ResourceHandle result = resourceId_++; map_.insert(std::pair(result, holder)); return result; } void erase(ResourceHandle moduleId) { - const std::lock_guard lock(mtx_); map_.erase(moduleId); } TResource lookup(ResourceHandle moduleId) { - const std::lock_guard lock(mtx_); auto it = map_.find(moduleId); if (it != map_.end()) { return it->second; @@ -54,12 +54,10 @@ class ResourceMap { } void clear() { - const std::lock_guard lock(mtx_); map_.clear(); } size_t size() { - const std::lock_guard lock(mtx_); return map_.size(); } @@ -71,9 +69,7 @@ class ResourceMap { ResourceHandle resourceId_; // map from resource ids returned to Java and resource pointers - folly::F14FastMap map_; - - std::mutex mtx_; + std::unordered_map map_; }; } // namespace gluten diff --git a/cpp/core/utils/StringUtil.cc b/cpp/core/utils/StringUtil.cc index 3d9155c26aae..d6acb9e02126 100644 --- a/cpp/core/utils/StringUtil.cc +++ b/cpp/core/utils/StringUtil.cc @@ -24,6 +24,9 @@ #include "exception.h" std::vector gluten::splitByDelim(const std::string& s, const char delimiter) { + if (s.empty()) { + return {}; + } std::vector result; size_t start = 0; size_t end = s.find(delimiter); @@ -38,13 +41,16 @@ std::vector gluten::splitByDelim(const std::string& s, const char d return result; } -std::vector gluten::splitPaths(const std::string& s) { +std::vector gluten::splitPaths(const std::string& s, bool checkExists) { + if (s.empty()) { + return {}; + } auto splits = splitByDelim(s, ','); std::vector paths; for (auto i = 0; i < splits.size(); ++i) { if (!splits[i].empty()) { std::filesystem::path path(splits[i]); - if (!std::filesystem::exists(path)) { + if (checkExists && !std::filesystem::exists(path)) { throw gluten::GlutenException("File path not exists: " + splits[i]); } if (path.is_relative()) { diff --git a/cpp/core/utils/StringUtil.h b/cpp/core/utils/StringUtil.h index f06cfa75251f..8880229616a5 100644 --- a/cpp/core/utils/StringUtil.h +++ b/cpp/core/utils/StringUtil.h @@ -21,6 +21,6 @@ namespace gluten { std::vector splitByDelim(const std::string& s, const char delimiter); -std::vector splitPaths(const std::string& s); +std::vector splitPaths(const std::string& s, bool checkExists = false); } // namespace gluten diff --git a/cpp/core/utils/macros.h b/cpp/core/utils/macros.h index 091dbbe8de9d..6fd0f15c0454 100644 --- a/cpp/core/utils/macros.h +++ b/cpp/core/utils/macros.h @@ -99,14 +99,6 @@ } \ std::cout << std::endl; -#define THROW_NOT_OK(expr) \ - do { \ - auto __s = (expr); \ - if (!__s.ok()) { \ - throw GlutenException(__s.message()); \ - } \ - } while (false); - #define TIME_TO_STRING(time) (time > 10000 ? time / 1000 : time) << (time > 10000 ? " ms" : " us") #define TIME_NANO_TO_STRING(time) \ diff --git a/cpp/velox/CMakeLists.txt b/cpp/velox/CMakeLists.txt index 2b425128109c..1ea9b38386a4 100644 --- a/cpp/velox/CMakeLists.txt +++ b/cpp/velox/CMakeLists.txt @@ -74,7 +74,24 @@ function(ADD_VELOX_DEPENDENCY VELOX_DEP_LIB_NAME VELOX_DEP_LIB_PATH) target_link_libraries(velox PUBLIC ${VELOX_DEP_LIB}) endfunction() +macro(ADD_VELOX_OBJECTS) + add_library(velox_objects OBJECT IMPORTED GLOBAL) + set_property(TARGET velox_objects PROPERTY IMPORTED_OBJECTS + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/FileHandle.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/HiveConfig.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/HiveConnector.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/HiveDataSink.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/HiveDataSource.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/HivePartitionUtil.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/PartitionIdGenerator.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/SplitReader.cpp.o" + "${VELOX_COMPONENTS_PATH}/connectors/hive/CMakeFiles/velox_hive_connector.dir/TableHandle.cpp.o" + ) + target_link_libraries(velox PUBLIC velox_objects) +endmacro() + macro(ADD_VELOX_DEPENDENCIES) + add_velox_objects() add_velox_dependency(functions::sparksql::lib "${VELOX_COMPONENTS_PATH}/functions/sparksql/libvelox_functions_spark.a") add_velox_dependency(functions::sparksql::agg "${VELOX_COMPONENTS_PATH}/functions/sparksql/aggregates/libvelox_functions_spark_aggregates.a") add_velox_dependency(functions::window::sparksql "${VELOX_COMPONENTS_PATH}/functions/sparksql/window/libvelox_functions_spark_window.a") @@ -84,19 +101,19 @@ macro(ADD_VELOX_DEPENDENCIES) add_velox_dependency(functions::lib::window "${VELOX_COMPONENTS_PATH}/functions/lib/window/libvelox_functions_window.a") add_velox_dependency(velox::buffer "${VELOX_COMPONENTS_PATH}/buffer/libvelox_buffer.a") - add_velox_dependency(exec "${VELOX_COMPONENTS_PATH}/exec/libvelox_exec.a") add_velox_dependency(functions::isnull "${VELOX_COMPONENTS_PATH}/functions/lib/libvelox_is_null_functions.a") add_velox_dependency(functions::prestosql "${VELOX_COMPONENTS_PATH}/functions/prestosql/registration/libvelox_functions_prestosql.a") add_velox_dependency(functions::prestosql::impl "${VELOX_COMPONENTS_PATH}/functions/prestosql/libvelox_functions_prestosql_impl.a") add_velox_dependency(functions::json "${VELOX_COMPONENTS_PATH}/functions/prestosql/json/libvelox_functions_json.a") add_velox_dependency(functions::hyperloglog "${VELOX_COMPONENTS_PATH}/common/hyperloglog/libvelox_common_hyperloglog.a") add_velox_dependency(functions::lib "${VELOX_COMPONENTS_PATH}/functions/lib/libvelox_functions_lib.a") - add_velox_dependency(common::test_util "${VELOX_COMPONENTS_PATH}/common/testutil/libvelox_test_util.a") if(BUILD_TESTS) add_velox_dependency(exec::test "${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_exec_test_lib.a") add_velox_dependency(temp::path "${VELOX_COMPONENTS_PATH}/exec/tests/utils/libvelox_temp_path.a") add_velox_dependency(dwio::common::test::utils "${VELOX_COMPONENTS_PATH}/dwio/common/tests/utils/libvelox_dwio_common_test_utils.a") endif() + add_velox_dependency(exec "${VELOX_COMPONENTS_PATH}/exec/libvelox_exec.a") + add_velox_dependency(common::test_util "${VELOX_COMPONENTS_PATH}/common/testutil/libvelox_test_util.a") add_velox_dependency(parse::parser "${VELOX_COMPONENTS_PATH}/parse/libvelox_parse_parser.a") if(BUILD_TESTS) add_velox_dependency(duckdb::parser "${VELOX_COMPONENTS_PATH}/duckdb/conversion/libvelox_duckdb_parser.a") @@ -108,39 +125,46 @@ macro(ADD_VELOX_DEPENDENCIES) endif() add_velox_dependency(vector::arrow::bridge "${VELOX_COMPONENTS_PATH}/vector/arrow/libvelox_arrow_bridge.a") add_velox_dependency(row "${VELOX_COMPONENTS_PATH}/row/libvelox_row_fast.a") - add_velox_dependency(connector::hive "${VELOX_COMPONENTS_PATH}/connectors/hive/libvelox_hive_connector.a") add_velox_dependency(connector "${VELOX_COMPONENTS_PATH}/connectors/libvelox_connector.a") add_velox_dependency(connector::hive_parition "${VELOX_COMPONENTS_PATH}/connectors/hive/libvelox_hive_partition_function.a") if(ENABLE_HDFS) add_velox_dependency(connector::hive::hdfs "${VELOX_COMPONENTS_PATH}/connectors/hive/storage_adapters/hdfs/libvelox_hdfs.a") endif() + if(ENABLE_GCS) + add_velox_dependency(connector::hive::gcs "${VELOX_COMPONENTS_PATH}/connectors/hive/storage_adapters/gcs/libvelox_gcs.a") + endif() if(ENABLE_S3) add_velox_dependency(connector::hive::s3fs "${VELOX_COMPONENTS_PATH}/connectors/hive/storage_adapters/s3fs/libvelox_s3fs.a") endif() - add_velox_dependency(dwio::dwrf::test_utils "${VELOX_COMPONENTS_PATH}/dwio/dwrf/test/utils/libvelox_dwrf_test_utils.a") add_velox_dependency(dwio::dwrf::writer "${VELOX_COMPONENTS_PATH}/dwio/dwrf/writer/libvelox_dwio_dwrf_writer.a") add_velox_dependency(dwio::dwrf::reader "${VELOX_COMPONENTS_PATH}/dwio/dwrf/reader/libvelox_dwio_dwrf_reader.a") add_velox_dependency(dwio::dwrf::utils "${VELOX_COMPONENTS_PATH}/dwio/dwrf/utils/libvelox_dwio_dwrf_utils.a") add_velox_dependency(dwio::dwrf::common "${VELOX_COMPONENTS_PATH}/dwio/dwrf/common/libvelox_dwio_dwrf_common.a") - add_velox_dependency(dwio::common::utils "${VELOX_COMPONENTS_PATH}/dwio/common/tests/utils/libvelox_dwio_common_test_utils.a") add_velox_dependency(parquet "${VELOX_COMPONENTS_PATH}/dwio/parquet/libvelox_dwio_parquet_reader.a") add_velox_dependency(parquet::reader::native "${VELOX_COMPONENTS_PATH}/dwio/parquet/reader/libvelox_dwio_native_parquet_reader.a") - add_velox_dependency(dwio::catalog::fbhive "${VELOX_COMPONENTS_PATH}/dwio/catalog/fbhive/libvelox_dwio_catalog_fbhive.a") - add_velox_dependency(parquet::reader::duckdb "${VELOX_COMPONENTS_PATH}/dwio/parquet/duckdb_reader/libvelox_dwio_duckdb_parquet_reader.a") - add_velox_dependency(parquet::reader::duckdb_allocator "${VELOX_COMPONENTS_PATH}/duckdb/memory/libvelox_duckdb_allocator.a") - add_velox_dependency(parquet::reader::duckdb_conversion "${VELOX_COMPONENTS_PATH}/duckdb/conversion/libvelox_duckdb_conversion.a") - add_velox_dependency(duckdb::duckdb "${VELOX_COMPONENTS_PATH}/external/duckdb/libduckdb.a") - add_velox_dependency(parquet::reader::duckdb_tpch_extension "${VELOX_COMPONENTS_PATH}/external/duckdb/tpch/libtpch_extension.a") - add_velox_dependency(dbgen "${VELOX_COMPONENTS_PATH}/external/duckdb/tpch/dbgen/libdbgen.a") + if(BUILD_TESTS) + add_velox_dependency(dwio::common::utils "${VELOX_COMPONENTS_PATH}/dwio/common/tests/utils/libvelox_dwio_common_test_utils.a") + add_velox_dependency(dwio::dwrf::test_utils "${VELOX_COMPONENTS_PATH}/dwio/dwrf/test/utils/libvelox_dwrf_test_utils.a") + add_velox_dependency(parquet::reader::duckdb_conversion "${VELOX_COMPONENTS_PATH}/duckdb/conversion/libvelox_duckdb_conversion.a") + add_velox_dependency(duckdb::duckdb "${VELOX_COMPONENTS_PATH}/external/duckdb/libduckdb.a") + add_velox_dependency(tpch::gen "${VELOX_COMPONENTS_PATH}/tpch/gen/libvelox_tpch_gen.a") + add_velox_dependency(dbgen "${VELOX_COMPONENTS_PATH}/tpch/gen/dbgen/libvelox_dbgen.a") + endif() add_velox_dependency(parquet::reader::thrift "${VELOX_COMPONENTS_PATH}/dwio/parquet/thrift/libvelox_dwio_parquet_thrift.a") - add_velox_dependency(dwio::type::fbhive "${VELOX_COMPONENTS_PATH}/dwio/type/fbhive/libvelox_dwio_type_fbhive.a") + add_velox_dependency(velox::arrow::parquet::writer "${VELOX_COMPONENTS_PATH}/dwio/parquet/writer/libvelox_dwio_arrow_parquet_writer.a") + add_velox_dependency(dwio::arrow::parquet::writer "${VELOX_COMPONENTS_PATH}/dwio/parquet/writer/arrow/libvelox_dwio_arrow_parquet_writer_lib.a") + add_velox_dependency(dwio::arrow::parquet::writer::util "${VELOX_COMPONENTS_PATH}/dwio/parquet/writer/arrow/util/libvelox_dwio_arrow_parquet_writer_util_lib.a") + add_velox_dependency(dwio::arrow::parquet::writer::thrift::lib "${VELOX_COMPONENTS_PATH}/dwio/parquet/writer/arrow/generated/libvelox_dwio_arrow_parquet_writer_thrift_lib.a") + add_velox_dependency(dwio::common::compression "${VELOX_COMPONENTS_PATH}/dwio/common/compression/libvelox_dwio_common_compression.a") add_velox_dependency(dwio::common "${VELOX_COMPONENTS_PATH}/dwio/common/libvelox_dwio_common.a") add_velox_dependency(functions::prestosql::types "${VELOX_COMPONENTS_PATH}/functions/prestosql/types/libvelox_presto_types.a") + add_velox_dependency(functions::spark::specialforms "${VELOX_COMPONENTS_PATH}/functions/sparksql/specialforms/libvelox_functions_spark_specialforms.a") add_velox_dependency(expression "${VELOX_COMPONENTS_PATH}/expression/libvelox_expression.a") add_velox_dependency(core "${VELOX_COMPONENTS_PATH}/core/libvelox_core.a") + add_velox_dependency(type::fbhive "${VELOX_COMPONENTS_PATH}/type/fbhive/libvelox_type_fbhive.a") add_velox_dependency(type "${VELOX_COMPONENTS_PATH}/type/libvelox_type.a") add_velox_dependency(vector::serializes "${VELOX_COMPONENTS_PATH}/serializers/libvelox_presto_serializer.a") add_velox_dependency(functions::lib::util "${VELOX_COMPONENTS_PATH}/functions/lib/libvelox_functions_util.a") @@ -150,18 +174,16 @@ macro(ADD_VELOX_DEPENDENCIES) add_velox_dependency(common::caching "${VELOX_COMPONENTS_PATH}/common/caching/libvelox_caching.a") add_velox_dependency(common::base "${VELOX_COMPONENTS_PATH}/common/base/libvelox_common_base.a") - if(BUILD_TESTS) - add_velox_dependency(tpch::gen "${VELOX_COMPONENTS_PATH}/tpch/gen/libvelox_tpch_gen.a") - endif() add_velox_dependency(common::memory "${VELOX_COMPONENTS_PATH}/common/memory/libvelox_memory.a") add_velox_dependency(common::serialization "${VELOX_COMPONENTS_PATH}/common/serialization/libvelox_serialization.a") + add_velox_dependency(spill::config "${VELOX_COMPONENTS_PATH}/common/config/libvelox_spill_config.a") add_velox_dependency(common::base::exception "${VELOX_COMPONENTS_PATH}/common/base/libvelox_exception.a") add_velox_dependency(type::tz "${VELOX_COMPONENTS_PATH}/type/tz/libvelox_type_tz.a") add_velox_dependency(dwio::dwrf::proto "${VELOX_COMPONENTS_PATH}/dwio/dwrf/proto/libvelox_dwio_dwrf_proto.a") + add_velox_dependency(dwio::catalog::fbhive "${VELOX_COMPONENTS_PATH}/dwio/catalog/fbhive/libvelox_dwio_catalog_fbhive.a") add_velox_dependency(dwio::common::exception "${VELOX_COMPONENTS_PATH}/dwio/common/exception/libvelox_dwio_common_exception.a") add_velox_dependency(dwio::common::encryption "${VELOX_COMPONENTS_PATH}/dwio/common/encryption/libvelox_dwio_common_encryption.a") - add_velox_dependency(dwio::common::compression "${VELOX_COMPONENTS_PATH}/dwio/common/compression/libvelox_dwio_common_compression.a") add_velox_dependency(core::config "${VELOX_COMPONENTS_PATH}/core/libvelox_config.a") add_velox_dependency(common::encode "${VELOX_COMPONENTS_PATH}/common/encode/libvelox_encode.a") @@ -172,11 +194,12 @@ macro(ADD_VELOX_DEPENDENCIES) add_velox_dependency(external::md5 "${VELOX_COMPONENTS_PATH}/external/md5/libmd5.a") add_velox_dependency(external::date "${VELOX_COMPONENTS_PATH}/external/date/libvelox_external_date.a") add_velox_dependency(velox::parquet::writer "${VELOX_COMPONENTS_PATH}/dwio/parquet/libvelox_dwio_parquet_writer.a") - add_velox_dependency(velox::arrow::parquet::writer "${VELOX_COMPONENTS_PATH}/dwio/parquet/writer/libvelox_dwio_arrow_parquet_writer.a") - + if(BUILD_TESTS) add_velox_dependency(vector::test::util "${VELOX_COMPONENTS_PATH}/vector/tests/utils/libvelox_vector_test_lib.a") endif() + add_velox_dependency(common::compression "${VELOX_COMPONENTS_PATH}/common/compression/libvelox_common_compression.a") + add_velox_dependency(common::io "${VELOX_COMPONENTS_PATH}/common/io/libvelox_common_io.a") endmacro() macro(find_libhdfs3) @@ -223,11 +246,17 @@ macro(find_awssdk) find_package(AWSSDK REQUIRED COMPONENTS s3;identity-management) endmacro() +macro(find_gcssdk) + set (CMAKE_FIND_LIBRARY_SUFFIXES ".so") + find_package(google_cloud_cpp_storage REQUIRED) +endmacro() + + # Build Velox backend. set(VELOX_SRCS compute/VeloxBackend.cc - compute/VeloxExecutionCtx.cc + compute/VeloxRuntime.cc compute/WholeStageResultIterator.cc compute/VeloxPlanConverter.cc jni/VeloxJniWrapper.cc @@ -236,18 +265,19 @@ set(VELOX_SRCS memory/VeloxColumnarBatch.cc memory/VeloxMemoryManager.cc operators/functions/RegistrationAllFunctions.cc + operators/functions/RowConstructorWithNull.cc operators/serializer/VeloxColumnarToRowConverter.cc operators/serializer/VeloxColumnarBatchSerializer.cc operators/serializer/VeloxRowToColumnarConverter.cc operators/writer/VeloxParquetDatasource.cc shuffle/VeloxShuffleReader.cc + shuffle/VeloxShuffleUtils.cc shuffle/VeloxShuffleWriter.cc substrait/SubstraitParser.cc substrait/SubstraitToVeloxExpr.cc substrait/SubstraitToVeloxPlan.cc substrait/SubstraitToVeloxPlanValidator.cc substrait/VariantToVectorConverter.cc - substrait/TypeUtils.cc substrait/SubstraitExtensionCollector.cc substrait/VeloxSubstraitSignature.cc substrait/VeloxToSubstraitExpr.cc @@ -258,6 +288,11 @@ set(VELOX_SRCS utils/ConfigExtractor.cc utils/Common.cc ) + +if(BUILD_TESTS OR BUILD_BENCHMARKS) + list(APPEND VELOX_SRCS utils/tests/MemoryPoolUtils.cc) +endif() + add_library(velox SHARED ${VELOX_SRCS}) if(ENABLE_GLUTEN_VCPKG) @@ -269,13 +304,9 @@ target_include_directories(velox PUBLIC ${CMAKE_SYSTEM_INCLUDE_PATH} ${JNI_INCLUDE_DIRS} ${CMAKE_CURRENT_SOURCE_DIR} - ${root_directory}/src - ${VELOX_HOME} - ${VELOX_BUILD_PATH} + ${VELOX_HOME}/ + ${VELOX_BUILD_PATH}/ ${VELOX_BUILD_PATH}/_deps/xsimd-src/include/ - ${VELOX_HOME}/velox/vector - ${VELOX_HOME}/velox/connectors - ${VELOX_HOME}/velox/external/xxhash/ ${VELOX_HOME}/third_party/xsimd/include/) set_target_properties(velox PROPERTIES @@ -287,23 +318,23 @@ set_target_properties(velox PROPERTIES ## It is also applicable to other dependencies. find_package(Folly REQUIRED CONFIG) -if(ENABLE_GLUTEN_VCPKG) - find_package(gflags REQUIRED COMPONENTS static CONFIG) -else() - find_package(gflags REQUIRED COMPONENTS shared CONFIG) -endif() - target_include_directories(velox PUBLIC ${GTEST_INCLUDE_DIRS} ${PROTOBUF_INCLUDE}) target_link_libraries(velox PUBLIC gluten) add_velox_dependencies() + +# Arrow libraries appear after Velox dependencies to avoid linker error +find_arrow_lib(${ARROW_LIB_NAME}) +find_arrow_lib(${PARQUET_LIB_NAME}) +target_link_libraries(velox PUBLIC Arrow::arrow Arrow::parquet) + target_link_libraries(velox PUBLIC Folly::folly) find_re2() target_link_libraries(velox PUBLIC ${RE2_LIBRARY}) -find_package(simdjson REQUIRED) +include(Simdjson) if(TARGET simdjson::simdjson AND NOT TARGET simdjson) add_library(simdjson INTERFACE) target_link_libraries(simdjson INTERFACE simdjson::simdjson) @@ -321,29 +352,35 @@ endif() if(Thrift_FOUND) target_link_libraries(velox PUBLIC thrift::thrift) else() - add_velox_dependency(thrift "${ARROW_HOME}/arrow_ep/cpp/build/thrift_ep-install/lib/libthrift.a") + add_velox_dependency(thrift "${ARROW_HOME}/src/arrow_ep-build/thrift_ep-install/lib/libthrift.a") endif() if(BUILD_TESTS) add_subdirectory(tests) endif() +if(BUILD_BENCHMARKS) + add_subdirectory(benchmarks) +endif() + if(ENABLE_HDFS) add_definitions(-DENABLE_HDFS) find_libhdfs3() target_link_libraries(velox PUBLIC HDFS::hdfs3) endif() -if(BUILD_BENCHMARKS) - add_subdirectory(benchmarks) -endif() - if(ENABLE_S3) add_definitions(-DENABLE_S3) find_awssdk() target_link_libraries(velox PUBLIC ${AWSSDK_LIBRARIES}) endif() +if(ENABLE_GCS) + add_definitions(-DENABLE_GCS) + find_gcssdk() + target_link_libraries(velox PUBLIC google-cloud-cpp::storage) +endif() + if(BUILD_EXAMPLES) add_subdirectory(udf/examples) endif() diff --git a/cpp/velox/benchmarks/BatchIteratorWrapper.h b/cpp/velox/benchmarks/BatchIteratorWrapper.h deleted file mode 100644 index 2539519c1a69..000000000000 --- a/cpp/velox/benchmarks/BatchIteratorWrapper.h +++ /dev/null @@ -1,111 +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 "BenchmarkUtils.h" - -#include "compute/ResultIterator.h" -#include "memory/ColumnarBatch.h" -#include "memory/ColumnarBatchIterator.h" -#include "utils/DebugOut.h" - -#include -#include -#include -#include -#include -#include - -namespace gluten { - -using GetInputFunc = std::shared_ptr(const std::string&); - -class BatchIterator : public ColumnarBatchIterator { - public: - explicit BatchIterator(const std::string& path) : path_(path) {} - - virtual ~BatchIterator() = default; - - virtual void createReader() = 0; - - virtual std::shared_ptr getSchema() = 0; - - int64_t getCollectBatchTime() const { - return collectBatchTime_; - } - - protected: - int64_t collectBatchTime_ = 0; - std::string path_; -}; - -class ParquetBatchIterator : public BatchIterator { - public: - explicit ParquetBatchIterator(const std::string& path) : BatchIterator(getExampleFilePath(path)) {} - - void createReader() override { - 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, &fileReader_)); - GLUTEN_THROW_NOT_OK( - fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); - - auto schema = recordBatchReader_->schema(); - std::cout << "schema:\n" << schema->ToString() << std::endl; - } - - std::shared_ptr getSchema() override { - return recordBatchReader_->schema(); - } - - protected: - std::unique_ptr fileReader_; - std::shared_ptr recordBatchReader_; -}; - -class OrcBatchIterator : public BatchIterator { - public: - explicit OrcBatchIterator(const std::string& path) : BatchIterator(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_; -}; - -} // namespace gluten diff --git a/cpp/velox/benchmarks/BatchStreamIterator.h b/cpp/velox/benchmarks/BatchStreamIterator.h deleted file mode 100644 index 44f35384978e..000000000000 --- a/cpp/velox/benchmarks/BatchStreamIterator.h +++ /dev/null @@ -1,70 +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 "BatchIteratorWrapper.h" - -namespace gluten { - -class ParquetBatchStreamIterator final : public ParquetBatchIterator { - public: - explicit ParquetBatchStreamIterator(const std::string& path) : ParquetBatchIterator(path) { - createReader(); - DEBUG_OUT << "ParquetBatchStreamIterator open file: " << path << std::endl; - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DEBUG_OUT << "ParquetBatchStreamIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0) << std::endl; - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -inline std::shared_ptr getParquetInputFromBatchStream(const std::string& path) { - return std::make_shared(std::make_unique(path)); -} - -class OrcBatchStreamIterator final : public OrcBatchIterator { - public: - explicit OrcBatchStreamIterator(const std::string& path) : OrcBatchIterator(path) { - createReader(); - } - - std::shared_ptr next() override { - auto startTime = std::chrono::steady_clock::now(); - GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); - DEBUG_OUT << "OrcBatchStreamIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0) << std::endl; - collectBatchTime_ += - std::chrono::duration_cast(std::chrono::steady_clock::now() - startTime).count(); - if (batch == nullptr) { - return nullptr; - } - return convertBatch(std::make_shared(batch)); - } -}; - -inline std::shared_ptr getOrcInputFromBatchStream(const std::string& path) { - return std::make_shared(std::make_unique(path)); -} - -} // namespace gluten diff --git a/cpp/velox/benchmarks/CMakeLists.txt b/cpp/velox/benchmarks/CMakeLists.txt index 253396545a20..5e6f222bd9f3 100644 --- a/cpp/velox/benchmarks/CMakeLists.txt +++ b/cpp/velox/benchmarks/CMakeLists.txt @@ -15,10 +15,14 @@ find_arrow_lib(${PARQUET_LIB_NAME}) +set(VELOX_BENCHMARK_COMMON_SRCS common/FileReaderIterator.cc common/BenchmarkUtils.cc) +add_library(velox_benchmark_common STATIC ${VELOX_BENCHMARK_COMMON_SRCS}) +target_include_directories(velox_benchmark_common PUBLIC ${CMAKE_SOURCE_DIR}/velox ${CMAKE_SOURCE_DIR}/core) +target_link_libraries(velox_benchmark_common PUBLIC Arrow::parquet velox benchmark::benchmark google::glog simdjson) + function(add_velox_benchmark BM_EXEC BM_FILE) - add_executable(${BM_EXEC} ${BM_FILE} BenchmarkUtils.cc) - target_include_directories(${BM_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox ${CMAKE_SOURCE_DIR}/src) - target_link_libraries(${BM_EXEC} gflags Arrow::parquet velox benchmark::benchmark simdjson) + add_executable(${BM_EXEC} ${BM_FILE}) + target_link_libraries(${BM_EXEC} PRIVATE velox_benchmark_common) endfunction() # Query benchmark @@ -33,3 +37,6 @@ add_velox_benchmark(parquet_write_benchmark ParquetWriteBenchmark.cc) add_velox_benchmark(shuffle_split_benchmark ShuffleSplitBenchmark.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 48ca2b057794..c3736134afac 100644 --- a/cpp/velox/benchmarks/GenericBenchmark.cc +++ b/cpp/velox/benchmarks/GenericBenchmark.cc @@ -24,11 +24,10 @@ #include #include -#include "BatchStreamIterator.h" -#include "BatchVectorIterator.h" -#include "BenchmarkUtils.h" -#include "compute/VeloxExecutionCtx.h" +#include "benchmarks/common/BenchmarkUtils.h" +#include "benchmarks/common/FileReaderIterator.h" #include "compute/VeloxPlanConverter.h" +#include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" #include "shuffle/LocalPartitionWriter.h" #include "shuffle/VeloxShuffleWriter.h" @@ -40,7 +39,6 @@ using namespace gluten; namespace { DEFINE_bool(skip_input, false, "Skip specifying input files."); -DEFINE_bool(gen_orc_input, false, "Generate orc files from parquet as input files."); DEFINE_bool(with_shuffle, false, "Add shuffle split at end."); DEFINE_string(partitioning, "rr", "Short partitioning name. Valid options are rr, hash, range, single"); DEFINE_bool(zstd, false, "Use ZSTD as shuffle compression codec"); @@ -49,9 +47,6 @@ DEFINE_bool(qat_zstd, false, "Use QAT ZSTD as shuffle compression codec"); DEFINE_bool(iaa_gzip, false, "Use IAA GZIP as shuffle compression codec"); DEFINE_int32(shuffle_partitions, 200, "Number of shuffle split (reducer) partitions"); -static const std::string kParquetSuffix = ".parquet"; -static const std::string kOrcSuffix = ".orc"; - struct WriterMetrics { int64_t splitTime; int64_t evictTime; @@ -65,7 +60,7 @@ std::shared_ptr createShuffleWriter(VeloxMemoryManager* memo auto options = ShuffleWriterOptions::defaults(); options.memory_pool = memoryManager->getArrowMemoryPool(); - options.partitioning_name = FLAGS_partitioning; + options.partitioning = gluten::toPartitioning(FLAGS_partitioning); if (FLAGS_zstd) { options.codec_backend = CodecBackend::NONE; options.compression_type = arrow::Compression::ZSTD; @@ -80,6 +75,8 @@ std::shared_ptr createShuffleWriter(VeloxMemoryManager* memo options.compression_type = arrow::Compression::GZIP; } + GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(options)); + GLUTEN_ASSIGN_OR_THROW( auto shuffleWriter, VeloxShuffleWriter::create( @@ -118,7 +115,7 @@ auto BM_Generic = [](::benchmark::State& state, const std::string& substraitJsonFile, const std::vector& inputFiles, const std::unordered_map& conf, - GetInputFunc* getInputIterator) { + FileReaderType readerType) { // Pin each threads to different CPU# starting from 0 or --cpu. if (FLAGS_cpu != -1) { setCpu(FLAGS_cpu + state.thread_index()); @@ -126,7 +123,7 @@ auto BM_Generic = [](::benchmark::State& state, setCpu(state.thread_index()); } auto memoryManager = getDefaultMemoryManager(); - auto executionCtx = ExecutionCtx::create(kVeloxExecutionCtxKind); + auto runtime = Runtime::create(kVeloxRuntimeKind, conf); const auto& filePath = getExampleFilePath(substraitJsonFile); auto plan = getPlanFromFile(filePath); auto startTime = std::chrono::steady_clock::now(); @@ -135,23 +132,24 @@ auto BM_Generic = [](::benchmark::State& state, for (auto _ : state) { std::vector> inputIters; - std::vector inputItersRaw; + std::vector inputItersRaw; if (!inputFiles.empty()) { - std::transform(inputFiles.cbegin(), inputFiles.cend(), std::back_inserter(inputIters), getInputIterator); + for (const auto& input : inputFiles) { + inputIters.push_back(getInputIteratorFromFileReader(input, readerType)); + } std::transform( inputIters.begin(), inputIters.end(), std::back_inserter(inputItersRaw), [](std::shared_ptr iter) { - return static_cast(iter->getInputIter()); + return static_cast(iter->getInputIter()); }); } - executionCtx->parsePlan(reinterpret_cast(plan.data()), plan.size()); - auto iterHandle = - executionCtx->createResultIterator(memoryManager.get(), "/tmp/test-spill", std::move(inputIters), conf); - auto resultIter = executionCtx->getResultIterator(iterHandle); - auto veloxPlan = dynamic_cast(executionCtx)->getVeloxPlan(); + runtime->parsePlan(reinterpret_cast(plan.data()), plan.size(), {}); + auto resultIter = + runtime->createResultIterator(memoryManager.get(), "/tmp/test-spill", std::move(inputIters), conf); + auto veloxPlan = dynamic_cast(runtime)->getVeloxPlan(); if (FLAGS_with_shuffle) { int64_t shuffleWriteTime; TIME_NANO_START(shuffleWriteTime); @@ -197,7 +195,7 @@ auto BM_Generic = [](::benchmark::State& state, } collectBatchTime += - std::accumulate(inputItersRaw.begin(), inputItersRaw.end(), 0, [](int64_t sum, BatchIterator* iter) { + std::accumulate(inputItersRaw.begin(), inputItersRaw.end(), 0, [](int64_t sum, FileReaderIterator* iter) { return sum + iter->getCollectBatchTime(); }); @@ -207,7 +205,7 @@ auto BM_Generic = [](::benchmark::State& state, auto statsStr = facebook::velox::exec::printPlanWithStats(*planNode, task->taskStats(), true); std::cout << statsStr << std::endl; } - ExecutionCtx::release(executionCtx); + Runtime::release(runtime); auto endTime = std::chrono::steady_clock::now(); auto duration = std::chrono::duration_cast(endTime - startTime).count(); @@ -226,80 +224,6 @@ auto BM_Generic = [](::benchmark::State& state, writerMetrics.compressTime, benchmark::Counter::kAvgIterations, benchmark::Counter::OneK::kIs1000); }; -class OrcFileGuard { - public: - explicit OrcFileGuard(const std::vector& inputFiles) { - orcFiles_.resize(inputFiles.size()); - for (auto i = 0; i != inputFiles.size(); ++i) { - GLUTEN_ASSIGN_OR_THROW(orcFiles_[i], createOrcFile(inputFiles[i])); - } - } - - ~OrcFileGuard() { - for (auto& x : orcFiles_) { - std::filesystem::remove(x); - } - } - - const std::vector& getOrcFiles() { - return orcFiles_; - } - - private: - arrow::Result createOrcFile(const std::string& inputFile) { - ParquetBatchStreamIterator parquetIterator(inputFile); - - std::string outputFile = inputFile; - // Get the filename. - auto pos = inputFile.find_last_of("/"); - if (pos != std::string::npos) { - outputFile = inputFile.substr(pos + 1); - } - // If any suffix is found, replace it with ".orc" - pos = outputFile.find_first_of("."); - if (pos != std::string::npos) { - outputFile = outputFile.substr(0, pos) + kOrcSuffix; - } else { - return arrow::Status::Invalid("Invalid input file: " + inputFile); - } - outputFile = std::filesystem::current_path().string() + "/" + outputFile; - - std::shared_ptr outputStream; - ARROW_ASSIGN_OR_RAISE(outputStream, arrow::io::FileOutputStream::Open(outputFile)); - - auto writerOptions = arrow::adapters::orc::WriteOptions(); - auto maybeWriter = arrow::adapters::orc::ORCFileWriter::Open(outputStream.get(), writerOptions); - GLUTEN_THROW_NOT_OK(maybeWriter); - auto& writer = *maybeWriter; - - while (true) { - // 1. read from Parquet - auto cb = parquetIterator.next(); - if (cb == nullptr) { - break; - } - - auto arrowColumnarBatch = std::dynamic_pointer_cast(cb); - auto recordBatch = arrowColumnarBatch->getRecordBatch(); - - // 2. write to Orc - if (!(writer->Write(*recordBatch)).ok()) { - return arrow::Status::IOError("Write failed"); - } - } - - if (!(writer->Close()).ok()) { - return arrow::Status::IOError("Close failed"); - } - - std::cout << "Created orc file: " << outputFile << std::endl; - - return outputFile; - } - - std::vector orcFiles_; -}; - int main(int argc, char** argv) { ::benchmark::Initialize(&argc, argv); gflags::ParseCommandLineFlags(&argc, &argv, true); @@ -307,9 +231,9 @@ int main(int argc, char** argv) { std::string substraitJsonFile; std::vector inputFiles; std::unordered_map conf; - std::shared_ptr orcFileGuard; conf.insert({gluten::kSparkBatchSize, FLAGS_batch_size}); + conf.insert({kDebugModeEnabled, "true"}); initVeloxBackend(conf); try { @@ -339,19 +263,19 @@ int main(int argc, char** argv) { std::exit(EXIT_FAILURE); } -#define GENERIC_BENCHMARK(NAME, FUNC) \ - do { \ - auto* bm = ::benchmark::RegisterBenchmark(NAME, BM_Generic, substraitJsonFile, inputFiles, conf, FUNC) \ - ->MeasureProcessCPUTime() \ - ->UseRealTime(); \ - if (FLAGS_threads > 0) { \ - bm->Threads(FLAGS_threads); \ - } else { \ - bm->ThreadRange(1, std::thread::hardware_concurrency()); \ - } \ - if (FLAGS_iterations > 0) { \ - bm->Iterations(FLAGS_iterations); \ - } \ +#define GENERIC_BENCHMARK(NAME, READER_TYPE) \ + do { \ + auto* bm = ::benchmark::RegisterBenchmark(NAME, BM_Generic, substraitJsonFile, inputFiles, conf, READER_TYPE) \ + ->MeasureProcessCPUTime() \ + ->UseRealTime(); \ + if (FLAGS_threads > 0) { \ + bm->Threads(FLAGS_threads); \ + } else { \ + bm->ThreadRange(1, std::thread::hardware_concurrency()); \ + } \ + if (FLAGS_iterations > 0) { \ + bm->Iterations(FLAGS_iterations); \ + } \ } while (0) #if 0 @@ -364,15 +288,10 @@ int main(int argc, char** argv) { #endif if (FLAGS_skip_input) { - GENERIC_BENCHMARK("SkipInput", nullptr); - } else if (FLAGS_gen_orc_input) { - orcFileGuard = std::make_shared(inputFiles); - inputFiles = orcFileGuard->getOrcFiles(); - GENERIC_BENCHMARK("OrcInputFromBatchVector", getOrcInputFromBatchVector); - GENERIC_BENCHMARK("OrcInputFromBatchStream", getOrcInputFromBatchStream); + GENERIC_BENCHMARK("SkipInput", FileReaderType::kNone); } else { - GENERIC_BENCHMARK("ParquetInputFromBatchVector", getParquetInputFromBatchVector); - GENERIC_BENCHMARK("ParquetInputFromBatchStream", getParquetInputFromBatchStream); + GENERIC_BENCHMARK("InputFromBatchVector", FileReaderType::kBuffered); + GENERIC_BENCHMARK("InputFromBatchStream", FileReaderType::kStream); } ::benchmark::RunSpecifiedBenchmarks(); diff --git a/cpp/velox/benchmarks/ParquetWriteBenchmark.cc b/cpp/velox/benchmarks/ParquetWriteBenchmark.cc index 33e88267e039..3f0fb2fce0db 100644 --- a/cpp/velox/benchmarks/ParquetWriteBenchmark.cc +++ b/cpp/velox/benchmarks/ParquetWriteBenchmark.cc @@ -33,8 +33,8 @@ #include -#include "BenchmarkUtils.h" -#include "compute/VeloxExecutionCtx.h" +#include "benchmarks/common/BenchmarkUtils.h" +#include "compute/VeloxRuntime.h" #include "memory/ArrowMemoryPool.h" #include "memory/ColumnarBatch.h" #include "memory/VeloxMemoryManager.h" @@ -256,7 +256,7 @@ class GoogleBenchmarkVeloxParquetWriteCacheScanBenchmark : public GoogleBenchmar // reuse the ParquetWriteConverter for batches caused system % increase a lot auto fileName = "velox_parquet_write.parquet"; - auto executionCtx = ExecutionCtx::create(kVeloxExecutionCtxKind); + auto runtime = Runtime::create(kVeloxRuntimeKind); auto memoryManager = getDefaultMemoryManager(); auto veloxPool = memoryManager->getAggregateMemoryPool(); @@ -265,7 +265,7 @@ class GoogleBenchmarkVeloxParquetWriteCacheScanBenchmark : public GoogleBenchmar auto veloxParquetDatasource = std::make_unique( outputPath_ + "/" + fileName, veloxPool->addAggregateChild("writer_benchmark"), localSchema); - veloxParquetDatasource->init(executionCtx->getConfMap()); + veloxParquetDatasource->init(runtime->getConfMap()); auto start = std::chrono::steady_clock::now(); for (const auto& vector : vectors) { veloxParquetDatasource->write(vector); @@ -292,7 +292,7 @@ class GoogleBenchmarkVeloxParquetWriteCacheScanBenchmark : public GoogleBenchmar benchmark::Counter(initTime, benchmark::Counter::kAvgThreads, benchmark::Counter::OneK::kIs1000); state.counters["write_time"] = benchmark::Counter(writeTime, benchmark::Counter::kAvgThreads, benchmark::Counter::OneK::kIs1000); - ExecutionCtx::release(executionCtx); + Runtime::release(runtime); } }; diff --git a/cpp/velox/benchmarks/QueryBenchmark.cc b/cpp/velox/benchmarks/QueryBenchmark.cc index 7d26f08c591a..7070267036d5 100644 --- a/cpp/velox/benchmarks/QueryBenchmark.cc +++ b/cpp/velox/benchmarks/QueryBenchmark.cc @@ -16,9 +16,9 @@ */ #include -#include +#include -#include "BenchmarkUtils.h" +#include "benchmarks/common/BenchmarkUtils.h" #include "compute/VeloxPlanConverter.h" #include "memory/VeloxMemoryManager.h" #include "utils/VeloxArrowUtils.h" @@ -34,18 +34,15 @@ const std::string getFilePath(const std::string& fileName) { // Used by unit test and benchmark. std::shared_ptr getResultIterator( - std::shared_ptr veloxPool, - ExecutionCtx* executionCtx, + VeloxMemoryManager* memoryManager, + Runtime* runtime, const std::vector>& setScanInfos, std::shared_ptr& veloxPlan) { - auto ctxPool = veloxPool->addAggregateChild( - "query_benchmark_result_iterator", facebook::velox::memory::MemoryReclaimer::create()); - std::vector> inputIter; std::unordered_map sessionConf = {}; auto veloxPlanConverter = std::make_unique(inputIter, defaultLeafVeloxMemoryPool().get(), sessionConf); - veloxPlan = veloxPlanConverter->toVeloxPlan(executionCtx->getPlan()); + veloxPlan = veloxPlanConverter->toVeloxPlan(runtime->getPlan()); // In test, use setScanInfos to replace the one got from Substrait. std::vector> scanInfos; @@ -53,21 +50,20 @@ std::shared_ptr getResultIterator( std::vector streamIds; // Separate the scan ids and stream ids, and get the scan infos. - VeloxExecutionCtx::getInfoAndIds( + VeloxRuntime::getInfoAndIds( veloxPlanConverter->splitInfos(), veloxPlan->leafPlanNodeIds(), scanInfos, scanIds, streamIds); auto wholestageIter = std::make_unique( - ctxPool, + memoryManager, veloxPlan, scanIds, setScanInfos, streamIds, "/tmp/test-spill", - executionCtx->getConfMap(), - executionCtx->getSparkTaskInfo()); - auto iter = std::make_shared(std::move(wholestageIter), executionCtx); - auto handle = executionCtx->addResultIterator(iter); - return executionCtx->getResultIterator(handle); + runtime->getConfMap(), + runtime->getSparkTaskInfo()); + auto iter = std::make_shared(std::move(wholestageIter), runtime); + return iter; } auto BM = [](::benchmark::State& state, @@ -78,8 +74,7 @@ auto BM = [](::benchmark::State& state, auto plan = getPlanFromFile(filePath); auto memoryManager = getDefaultMemoryManager(); - auto executionCtx = ExecutionCtx::create(kVeloxExecutionCtxKind); - auto veloxPool = memoryManager->getAggregateMemoryPool(); + auto runtime = Runtime::create(kVeloxRuntimeKind); std::vector> scanInfos; scanInfos.reserve(datasetPaths.size()); @@ -95,9 +90,9 @@ auto BM = [](::benchmark::State& state, state.PauseTiming(); state.ResumeTiming(); - executionCtx->parsePlan(reinterpret_cast(plan.data()), plan.size()); + runtime->parsePlan(reinterpret_cast(plan.data()), plan.size(), {}); std::shared_ptr veloxPlan; - auto resultIter = getResultIterator(veloxPool, executionCtx, scanInfos, veloxPlan); + auto resultIter = getResultIterator(memoryManager.get(), runtime, scanInfos, veloxPlan); auto outputSchema = toArrowSchema(veloxPlan->outputType(), defaultLeafVeloxMemoryPool().get()); while (resultIter->hasNext()) { auto array = resultIter->next()->exportArrowArray(); @@ -109,7 +104,7 @@ auto BM = [](::benchmark::State& state, std::cout << maybeBatch.ValueOrDie()->ToString() << std::endl; } } - ExecutionCtx::release(executionCtx); + Runtime::release(runtime); }; #define orc_reader_decimal 1 diff --git a/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc b/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc index d5b718dbcb5d..c4aec9650e1b 100644 --- a/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc +++ b/cpp/velox/benchmarks/ShuffleSplitBenchmark.cc @@ -28,7 +28,7 @@ #include -#include "benchmarks/BenchmarkUtils.h" +#include "benchmarks/common/BenchmarkUtils.h" #include "memory/ColumnarBatch.h" #include "shuffle/LocalPartitionWriter.h" #include "shuffle/VeloxShuffleWriter.h" @@ -115,9 +115,9 @@ class BenchmarkShuffleSplit { auto options = ShuffleWriterOptions::defaults(); options.buffer_size = kPartitionBufferSize; - options.buffered_write = true; options.memory_pool = pool.get(); - options.partitioning_name = "rr"; + options.partitioning = Partitioning::kRoundRobin; + GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFileFromEnv(options)); std::shared_ptr shuffleWriter; int64_t elapseRead = 0; diff --git a/cpp/velox/benchmarks/BenchmarkUtils.cc b/cpp/velox/benchmarks/common/BenchmarkUtils.cc similarity index 77% rename from cpp/velox/benchmarks/BenchmarkUtils.cc rename to cpp/velox/benchmarks/common/BenchmarkUtils.cc index 1c973ec2423a..00dc804389fe 100644 --- a/cpp/velox/benchmarks/BenchmarkUtils.cc +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.cc @@ -17,8 +17,10 @@ #include "BenchmarkUtils.h" #include "compute/VeloxBackend.h" -#include "compute/VeloxExecutionCtx.h" +#include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" +#include "shuffle/Utils.h" +#include "utils/StringUtil.h" #include "velox/dwio/common/Options.h" using namespace facebook; @@ -35,14 +37,14 @@ namespace { std::unordered_map bmConfMap = {{gluten::kSparkBatchSize, FLAGS_batch_size}}; -gluten::ExecutionCtx* veloxExecutionCtxFactory(const std::unordered_map& sparkConf) { - return new gluten::VeloxExecutionCtx(sparkConf); +gluten::Runtime* veloxRuntimeFactory(const std::unordered_map& sparkConf) { + return new gluten::VeloxRuntime(sparkConf); } -} // anonymous namespace +} // namespace void initVeloxBackend(std::unordered_map& conf) { - gluten::ExecutionCtx::registerFactory(gluten::kVeloxExecutionCtxKind, veloxExecutionCtxFactory); + gluten::Runtime::registerFactory(gluten::kVeloxRuntimeKind, veloxRuntimeFactory); gluten::VeloxBackend::create(conf); } @@ -150,3 +152,27 @@ void setCpu(uint32_t cpuindex) { exit(EXIT_FAILURE); } } + +arrow::Status setLocalDirsAndDataFileFromEnv(gluten::ShuffleWriterOptions& options) { + auto joinedDirsC = std::getenv(gluten::kGlutenSparkLocalDirs.c_str()); + if (joinedDirsC != nullptr && strcmp(joinedDirsC, "") > 0) { + // Set local dirs. + auto joinedDirs = std::string(joinedDirsC); + options.local_dirs = joinedDirs; + // Split local dirs and use thread id to choose one directory for data file. + auto localDirs = gluten::splitPaths(joinedDirs); + size_t id = std::hash{}(std::this_thread::get_id()) % localDirs.size(); + ARROW_ASSIGN_OR_RAISE(options.data_file, gluten::createTempShuffleFile(localDirs[id])); + } else { + // 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)) + options.local_dirs = tmpDir->path().ToString(); + } + ARROW_ASSIGN_OR_RAISE(options.data_file, gluten::createTempShuffleFile(options.local_dirs)); + } + return arrow::Status::OK(); +} diff --git a/cpp/velox/benchmarks/BenchmarkUtils.h b/cpp/velox/benchmarks/common/BenchmarkUtils.h similarity index 95% rename from cpp/velox/benchmarks/BenchmarkUtils.h rename to cpp/velox/benchmarks/common/BenchmarkUtils.h index 8b8fd84835ad..16a22cdb1d50 100644 --- a/cpp/velox/benchmarks/BenchmarkUtils.h +++ b/cpp/velox/benchmarks/common/BenchmarkUtils.h @@ -17,17 +17,19 @@ #pragma once -#include +#include #include #include #include #include #include +#include "benchmark/benchmark.h" #include "substrait/SubstraitToVeloxPlan.h" #include "compute/ProtobufUtils.h" #include "memory/VeloxColumnarBatch.h" #include "memory/VeloxMemoryManager.h" +#include "shuffle/Options.h" #include "utils/exception.h" #include "velox/common/memory/Memory.h" #include "velox/dwio/common/tests/utils/DataFiles.h" @@ -103,4 +105,6 @@ inline std::shared_ptr convertBatch(std::shared_ptr gluten::getInputIteratorFromFileReader( + const std::string& path, + gluten::FileReaderType readerType) { + 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)); + } + if (readerType == FileReaderType::kBuffered) { + return std::make_shared(std::make_unique(path)); + } + } 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."); +} diff --git a/cpp/velox/benchmarks/common/FileReaderIterator.h b/cpp/velox/benchmarks/common/FileReaderIterator.h new file mode 100644 index 000000000000..c12c90e22706 --- /dev/null +++ b/cpp/velox/benchmarks/common/FileReaderIterator.h @@ -0,0 +1,59 @@ +/* + * 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 "BenchmarkUtils.h" +#include "compute/ResultIterator.h" +#include "memory/ColumnarBatch.h" +#include "memory/ColumnarBatchIterator.h" +#include "utils/DebugOut.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) {} + + virtual ~FileReaderIterator() = default; + + virtual void createReader() = 0; + + virtual std::shared_ptr getSchema() = 0; + + int64_t getCollectBatchTime() const { + return collectBatchTime_; + } + + 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/benchmarks/BatchVectorIterator.h b/cpp/velox/benchmarks/common/OrcReaderIterator.h similarity index 53% rename from cpp/velox/benchmarks/BatchVectorIterator.h rename to cpp/velox/benchmarks/common/OrcReaderIterator.h index 30152bb3d315..8bc2a50c9ec3 100644 --- a/cpp/velox/benchmarks/BatchVectorIterator.h +++ b/cpp/velox/benchmarks/common/OrcReaderIterator.h @@ -16,57 +16,70 @@ */ #pragma once -#include "BatchIteratorWrapper.h" +#include +#include "benchmarks/common/FileReaderIterator.h" namespace gluten { -class ParquetBatchVectorIterator final : public ParquetBatchIterator { +class OrcReaderIterator : public FileReaderIterator { public: - explicit ParquetBatchVectorIterator(const std::string& path) : ParquetBatchIterator(path) { - createReader(); - collectBatches(); + explicit OrcReaderIterator(const std::string& path) : FileReaderIterator(path) {} - iter_ = batches_.begin(); - DEBUG_OUT << "ParquetBatchVectorIterator open file: " << path << std::endl; - DEBUG_OUT << "Number of input batches: " << std::to_string(batches_.size()) << std::endl; - if (iter_ != batches_.cend()) { - DEBUG_OUT << "columns: " << (*iter_)->num_columns() << std::endl; - DEBUG_OUT << "rows: " << (*iter_)->num_rows() << std::endl; - } - } + void createReader() override { + // Open File + auto input = arrow::io::ReadableFile::Open(path_); + GLUTEN_THROW_NOT_OK(input); - std::shared_ptr next() override { - if (iter_ == batches_.cend()) { - return nullptr; - } - return convertBatch(std::make_shared(*iter_++)); + // 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; } - 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(); + std::shared_ptr getSchema() override { + auto schema = fileReader_->ReadSchema(); + GLUTEN_THROW_NOT_OK(schema); + return *schema; } - arrow::RecordBatchVector batches_; - std::vector>::const_iterator iter_; + protected: + std::unique_ptr fileReader_; + std::shared_ptr recordBatchReader_; }; -inline std::shared_ptr getParquetInputFromBatchVector(const std::string& path) { - return std::make_shared(std::make_unique(path)); -} +class OrcStreamReaderIterator final : public OrcReaderIterator { + public: + explicit OrcStreamReaderIterator(const std::string& path) : OrcReaderIterator(path) { + createReader(); + } -class OrcBatchVectorIterator final : public OrcBatchIterator { + std::shared_ptr next() override { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); + DEBUG_OUT << "OrcStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0) << std::endl; + 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 OrcBatchVectorIterator(const std::string& path) : OrcBatchIterator(path) { + explicit OrcBufferedReaderIterator(const std::string& path) : OrcReaderIterator(path) { createReader(); collectBatches(); iter_ = batches_.begin(); #ifdef GLUTEN_PRINT_DEBUG - DEBUG_OUT << "OrcBatchVectorIterator open file: " << path << std::endl; + DEBUG_OUT << "OrcBufferedReaderIterator open file: " << path << std::endl; DEBUG_OUT << "Number of input batches: " << std::to_string(batches_.size()) << std::endl; if (iter_ != batches_.cend()) { DEBUG_OUT << "columns: " << (*iter_)->num_columns() << std::endl; @@ -94,8 +107,4 @@ class OrcBatchVectorIterator final : public OrcBatchIterator { std::vector>::const_iterator iter_; }; -inline std::shared_ptr getOrcInputFromBatchVector(const std::string& path) { - return std::make_shared(std::make_unique(path)); -} - -} // namespace gluten +} // namespace gluten \ No newline at end of file diff --git a/cpp/velox/benchmarks/common/ParquetReaderIterator.h b/cpp/velox/benchmarks/common/ParquetReaderIterator.h new file mode 100644 index 000000000000..f6621719786a --- /dev/null +++ b/cpp/velox/benchmarks/common/ParquetReaderIterator.h @@ -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. + */ + +#include "benchmarks/common/FileReaderIterator.h" +#include "utils/macros.h" + +#include + +namespace gluten { + +class ParquetReaderIterator : public FileReaderIterator { + public: + explicit ParquetReaderIterator(const std::string& path) : FileReaderIterator(getExampleFilePath(path)) {} + + void createReader() override { + 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, &fileReader_)); + GLUTEN_THROW_NOT_OK( + fileReader_->GetRecordBatchReader(arrow::internal::Iota(fileReader_->num_row_groups()), &recordBatchReader_)); + + auto schema = recordBatchReader_->schema(); + std::cout << "schema:\n" << schema->ToString() << std::endl; + } + + 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(); + DEBUG_OUT << "ParquetStreamReaderIterator open file: " << path << std::endl; + } + + std::shared_ptr next() override { + auto startTime = std::chrono::steady_clock::now(); + GLUTEN_ASSIGN_OR_THROW(auto batch, recordBatchReader_->Next()); + DEBUG_OUT << "ParquetStreamReaderIterator get a batch, num rows: " << (batch ? batch->num_rows() : 0) << std::endl; + 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(); + DEBUG_OUT << "ParquetBufferedReaderIterator open file: " << path << std::endl; + DEBUG_OUT << "Number of input batches: " << std::to_string(batches_.size()) << std::endl; + if (iter_ != batches_.cend()) { + DEBUG_OUT << "columns: " << (*iter_)->num_columns() << std::endl; + DEBUG_OUT << "rows: " << (*iter_)->num_rows() << std::endl; + } + } + + 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 new file mode 100644 index 000000000000..b421ecca3b37 --- /dev/null +++ b/cpp/velox/benchmarks/exec/OrcConverter.cc @@ -0,0 +1,107 @@ +/* + * 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 "benchmarks/common/ParquetReaderIterator.h" + +namespace gluten { + +class OrcConverter final { + public: + explicit OrcConverter(const std::vector& inputFiles) : inputFiles_(inputFiles) { + orcFiles_.resize(inputFiles.size()); + } + + const std::vector& getOrcFiles() { + for (auto i = 0; i != inputFiles_.size(); ++i) { + GLUTEN_ASSIGN_OR_THROW(orcFiles_[i], createOrcFile(inputFiles_[i])); + } + return orcFiles_; + } + + private: + arrow::Result createOrcFile(const std::string& inputFile) { + ParquetStreamReaderIterator parquetIterator(inputFile); + + std::string outputFile = inputFile; + // Get the filename. + auto pos = inputFile.find_last_of("/"); + if (pos != std::string::npos) { + outputFile = inputFile.substr(pos + 1); + } + // If any suffix is found, replace it with ".orc" + pos = outputFile.find_first_of("."); + if (pos != std::string::npos) { + outputFile = outputFile.substr(0, pos) + kOrcSuffix; + } else { + return arrow::Status::Invalid("Invalid input file: " + inputFile); + } + outputFile = std::filesystem::current_path().string() + "/" + outputFile; + + std::shared_ptr outputStream; + ARROW_ASSIGN_OR_RAISE(outputStream, arrow::io::FileOutputStream::Open(outputFile)); + + auto writerOptions = arrow::adapters::orc::WriteOptions(); + auto maybeWriter = arrow::adapters::orc::ORCFileWriter::Open(outputStream.get(), writerOptions); + GLUTEN_THROW_NOT_OK(maybeWriter); + auto& writer = *maybeWriter; + + // Read from parquet and write to ORC. + while (auto cb = parquetIterator.next()) { + GLUTEN_ASSIGN_OR_THROW( + auto recordBatch, arrow::ImportRecordBatch(cb->exportArrowArray().get(), parquetIterator.getSchema())); + if (!(writer->Write(*recordBatch)).ok()) { + return arrow::Status::IOError("Write failed"); + } + } + + if (!(writer->Close()).ok()) { + return arrow::Status::IOError("Close failed"); + } + return outputFile; + } + + std::vector inputFiles_; + std::vector orcFiles_; +}; + +} // namespace gluten + +int main(int argc, char** argv) { + if (argc == 1) { + std::cout << "Please specify parquet files as input arguments." << std::endl; + exit(0); + } + + std::vector inputFiles; + for (auto i = 1; i < argc; ++i) { + const auto& file = argv[i]; + if (!std::filesystem::exists(file)) { + std::cout << file << " doesn't exist!" << std::endl; + exit(1); + } + inputFiles.emplace_back(argv[i]); + } + + auto orcConverter = std::make_shared(inputFiles); + auto orcFiles = orcConverter->getOrcFiles(); + std::cout << "Generated output files: " << std::endl; + for (const auto& file : orcFiles) { + std::cout << file << std::endl; + } + return 0; +} diff --git a/cpp/velox/compute/VeloxBackend.cc b/cpp/velox/compute/VeloxBackend.cc index a5700302ccbe..ad7f8fc36bc8 100644 --- a/cpp/velox/compute/VeloxBackend.cc +++ b/cpp/velox/compute/VeloxBackend.cc @@ -31,27 +31,25 @@ #ifdef GLUTEN_ENABLE_IAA #include "utils/qpl/qpl_codec.h" #endif -#include "utils/exception.h" -#include "velox/common/file/FileSystems.h" -#include "velox/serializers/PrestoSerializer.h" -#ifdef ENABLE_HDFS -#include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" -#endif -#ifdef ENABLE_S3 -#include "velox/connectors/hive/storage_adapters/s3fs/S3FileSystem.h" +#ifdef ENABLE_GCS +#include #endif +#include "config/GlutenConfig.h" #include "jni/JniFileSystem.h" #include "udf/UdfLoader.h" #include "utils/ConfigExtractor.h" +#include "utils/exception.h" +#include "velox/common/caching/SsdCache.h" +#include "velox/common/file/FileSystems.h" #include "velox/common/memory/MmapAllocator.h" +#include "velox/connectors/hive/HiveConfig.h" #include "velox/connectors/hive/HiveConnector.h" -#include "velox/dwio/dwrf/reader/DwrfReader.h" -#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/serializers/PrestoSerializer.h" DECLARE_int32(split_preload_per_driver); -DECLARE_bool(SkipRowSortInWindowOp); DECLARE_bool(velox_exception_user_stacktrace_enabled); DECLARE_int32(velox_memory_num_shared_leaf_pools); +DECLARE_bool(velox_memory_use_hugepages); using namespace facebook; @@ -59,42 +57,51 @@ namespace { const std::string kEnableUserExceptionStacktrace = "spark.gluten.sql.columnar.backend.velox.enableUserExceptionStacktrace"; -const std::string kEnableUserExceptionStacktraceDefault = "true"; +const bool kEnableUserExceptionStacktraceDefault = true; + +const std::string kGlogVerboseLevel = "spark.gluten.sql.columnar.backend.velox.glogVerboseLevel"; +const uint32_t kGlogVerboseLevelDefault = 0; + +const std::string kGlogSeverityLevel = "spark.gluten.sql.columnar.backend.velox.glogSeverityLevel"; +const uint32_t kGlogSeverityLevelDefault = 0; const std::string kEnableSystemExceptionStacktrace = "spark.gluten.sql.columnar.backend.velox.enableSystemExceptionStacktrace"; -const std::string kEnableSystemExceptionStacktraceDefault = "true"; +const bool kEnableSystemExceptionStacktraceDefault = true; + +const std::string kMemoryUseHugePages = "spark.gluten.sql.columnar.backend.velox.memoryUseHugePages"; +const bool kMemoryUseHugePagesDefault = false; const std::string kHiveConnectorId = "test-hive"; const std::string kVeloxCacheEnabled = "spark.gluten.sql.columnar.backend.velox.cacheEnabled"; // memory cache const std::string kVeloxMemCacheSize = "spark.gluten.sql.columnar.backend.velox.memCacheSize"; -const std::string kVeloxMemCacheSizeDefault = "1073741824"; +const uint64_t kVeloxMemCacheSizeDefault = 1073741824; // 1G // ssd cache const std::string kVeloxSsdCacheSize = "spark.gluten.sql.columnar.backend.velox.ssdCacheSize"; -const std::string kVeloxSsdCacheSizeDefault = "1073741824"; +const uint64_t kVeloxSsdCacheSizeDefault = 1073741824; // 1G const std::string kVeloxSsdCachePath = "spark.gluten.sql.columnar.backend.velox.ssdCachePath"; const std::string kVeloxSsdCachePathDefault = "/tmp/"; const std::string kVeloxSsdCacheShards = "spark.gluten.sql.columnar.backend.velox.ssdCacheShards"; -const std::string kVeloxSsdCacheShardsDefault = "1"; +const uint32_t kVeloxSsdCacheShardsDefault = 1; const std::string kVeloxSsdCacheIOThreads = "spark.gluten.sql.columnar.backend.velox.ssdCacheIOThreads"; -const std::string kVeloxSsdCacheIOThreadsDefault = "1"; +const uint32_t kVeloxSsdCacheIOThreadsDefault = 1; const std::string kVeloxSsdODirectEnabled = "spark.gluten.sql.columnar.backend.velox.ssdODirect"; const std::string kVeloxIOThreads = "spark.gluten.sql.columnar.backend.velox.IOThreads"; -const std::string kVeloxIOThreadsDefault = "0"; +const uint32_t kVeloxIOThreadsDefault = 0; const std::string kVeloxSplitPreloadPerDriver = "spark.gluten.sql.columnar.backend.velox.SplitPreloadPerDriver"; -const std::string kVeloxSplitPreloadPerDriverDefault = "2"; +const uint32_t kVeloxSplitPreloadPerDriverDefault = 2; // udf const std::string kVeloxUdfLibraryPaths = "spark.gluten.sql.columnar.backend.velox.udfLibraryPaths"; // spill const std::string kMaxSpillFileSize = "spark.gluten.sql.columnar.backend.velox.maxSpillFileSize"; -const std::string kMaxSpillFileSizeDefault = std::to_string(20L * 1024 * 1024); +const uint64_t kMaxSpillFileSizeDefault = 20L * 1024 * 1024; // backtrace allocation const std::string kBacktraceAllocation = "spark.gluten.backtrace.allocation"; @@ -102,83 +109,58 @@ const std::string kBacktraceAllocation = "spark.gluten.backtrace.allocation"; // VeloxShuffleReader print flag. const std::string kVeloxShuffleReaderPrintFlag = "spark.gluten.velox.shuffleReaderPrintFlag"; +const std::string kVeloxFileHandleCacheEnabled = "spark.gluten.sql.columnar.backend.velox.fileHandleCacheEnabled"; +const bool kVeloxFileHandleCacheEnabledDefault = false; + } // namespace namespace gluten { -void VeloxBackend::printConf(const std::unordered_map& conf) { - std::ostringstream oss; - oss << "STARTUP: VeloxBackend conf = {\n"; - for (auto& [k, v] : conf) { - oss << " {" << k << ", " << v << "}\n"; - } - oss << "}\n"; - LOG(INFO) << oss.str(); -} - void VeloxBackend::init(const std::unordered_map& conf) { - // In spark, planner takes care the partitioning and sorting, so the rows are sorted. - // There is no need to sort the rows in window op again. - FLAGS_SkipRowSortInWindowOp = true; + // Init glog and log level. + auto veloxmemcfg = std::make_shared(conf); + const facebook::velox::Config* veloxcfg = veloxmemcfg.get(); + + uint32_t vlogLevel = veloxcfg->get(kGlogVerboseLevel, kGlogVerboseLevelDefault); + uint32_t severityLogLevel = veloxcfg->get(kGlogSeverityLevel, kGlogSeverityLevelDefault); + FLAGS_v = vlogLevel; + FLAGS_minloglevel = severityLogLevel; + FLAGS_logtostderr = true; + google::InitGoogleLogging("gluten"); + // Avoid creating too many shared leaf pools. FLAGS_velox_memory_num_shared_leaf_pools = 0; // Set velox_exception_user_stacktrace_enabled. - { - auto got = conf.find(kEnableUserExceptionStacktrace); - std::string enableUserExceptionStacktrace = kEnableUserExceptionStacktraceDefault; - if (got != conf.end()) { - enableUserExceptionStacktrace = got->second; - } - FLAGS_velox_exception_user_stacktrace_enabled = (enableUserExceptionStacktrace == "true"); - } + FLAGS_velox_exception_user_stacktrace_enabled = + veloxcfg->get(kEnableUserExceptionStacktrace, kEnableUserExceptionStacktraceDefault); // Set velox_exception_system_stacktrace_enabled. - { - auto got = conf.find(kEnableSystemExceptionStacktrace); - std::string enableSystemExceptionStacktrace = kEnableSystemExceptionStacktraceDefault; - if (got != conf.end()) { - enableSystemExceptionStacktrace = got->second; - } - FLAGS_velox_exception_system_stacktrace_enabled = (enableSystemExceptionStacktrace == "true"); - } + FLAGS_velox_exception_system_stacktrace_enabled = + veloxcfg->get(kEnableSystemExceptionStacktrace, kEnableSystemExceptionStacktraceDefault); + + // Set velox_memory_use_hugepages. + FLAGS_velox_memory_use_hugepages = veloxcfg->get(kMemoryUseHugePages, kMemoryUseHugePagesDefault); // Set backtrace_allocation - { - auto got = conf.find(kBacktraceAllocation); - if (got != conf.end()) { - gluten::backtrace_allocation = (got->second == "true"); - } - } + gluten::backtrace_allocation = veloxcfg->get(kBacktraceAllocation, false); // Set veloxShuffleReaderPrintFlag - { - auto got = conf.find(kVeloxShuffleReaderPrintFlag); - if (got != conf.end()) { - gluten::veloxShuffleReaderPrintFlag = (got->second == "true"); - } - } + gluten::veloxShuffleReaderPrintFlag = veloxcfg->get(kVeloxShuffleReaderPrintFlag, false); // Setup and register. velox::filesystems::registerLocalFileSystem(); - initJolFilesystem(conf); + initJolFilesystem(veloxcfg); -#ifdef ENABLE_HDFS - velox::filesystems::registerHdfsFileSystem(); -#endif - - std::unordered_map configurationValues; #ifdef ENABLE_S3 - velox::filesystems::registerS3FileSystem(); - - std::string awsAccessKey = conf.at("spark.hadoop.fs.s3a.access.key"); - std::string awsSecretKey = conf.at("spark.hadoop.fs.s3a.secret.key"); - std::string awsEndpoint = conf.at("spark.hadoop.fs.s3a.endpoint"); - std::string sslEnabled = conf.at("spark.hadoop.fs.s3a.connection.ssl.enabled"); - std::string pathStyleAccess = conf.at("spark.hadoop.fs.s3a.path.style.access"); - std::string useInstanceCredentials = conf.at("spark.hadoop.fs.s3a.use.instance.credentials"); - std::string iamRole = conf.at("spark.hadoop.fs.s3a.iam.role"); - std::string iamRoleSessionName = conf.at("spark.hadoop.fs.s3a.iam.role.session.name"); + std::string awsAccessKey = veloxcfg->get("spark.hadoop.fs.s3a.access.key", ""); + std::string awsSecretKey = veloxcfg->get("spark.hadoop.fs.s3a.secret.key", ""); + std::string awsEndpoint = veloxcfg->get("spark.hadoop.fs.s3a.endpoint", ""); + bool sslEnabled = veloxcfg->get("spark.hadoop.fs.s3a.connection.ssl.enabled", false); + bool pathStyleAccess = veloxcfg->get("spark.hadoop.fs.s3a.path.style.access", false); + bool useInstanceCredentials = veloxcfg->get("spark.hadoop.fs.s3a.use.instance.credentials", false); + std::string iamRole = veloxcfg->get("spark.hadoop.fs.s3a.iam.role", ""); + std::string iamRoleSessionName = veloxcfg->get("spark.hadoop.fs.s3a.iam.role.session.name", ""); const char* envAwsAccessKey = std::getenv("AWS_ACCESS_KEY_ID"); if (envAwsAccessKey != nullptr) { @@ -194,56 +176,87 @@ void VeloxBackend::init(const std::unordered_map& conf } std::unordered_map s3Config({}); - if (useInstanceCredentials == "true") { - s3Config.insert({ - {"hive.s3.use-instance-credentials", useInstanceCredentials}, - }); + if (useInstanceCredentials) { + veloxmemcfg->setValue("hive.s3.use-instance-credentials", "true"); } else if (!iamRole.empty()) { - s3Config.insert({ - {"hive.s3.iam-role", iamRole}, - }); + veloxmemcfg->setValue("hive.s3.iam-role", iamRole); if (!iamRoleSessionName.empty()) { - s3Config.insert({ - {"hive.s3.iam-role-session-name", iamRoleSessionName}, - }); + veloxmemcfg->setValue("hive.s3.iam-role-session-name", iamRoleSessionName); } } else { - s3Config.insert({ - {"hive.s3.aws-access-key", awsAccessKey}, - {"hive.s3.aws-secret-key", awsSecretKey}, - }); + veloxmemcfg->setValue("hive.s3.aws-access-key", awsAccessKey); + veloxmemcfg->setValue("hive.s3.aws-secret-key", awsSecretKey); } // Only need to set s3 endpoint when not use instance credentials. - if (useInstanceCredentials != "true") { - s3Config.insert({ - {"hive.s3.endpoint", awsEndpoint}, - }); + if (!useInstanceCredentials) { + veloxmemcfg->setValue("hive.s3.endpoint", awsEndpoint); } - s3Config.insert({ - {"hive.s3.ssl.enabled", sslEnabled}, - {"hive.s3.path-style-access", pathStyleAccess}, - }); - - configurationValues.merge(s3Config); + veloxmemcfg->setValue("hive.s3.ssl.enabled", sslEnabled ? "true" : "false"); + veloxmemcfg->setValue("hive.s3.path-style-access", pathStyleAccess ? "true" : "false"); #endif +#ifdef ENABLE_GCS + // https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/CONFIGURATION.md#api-client-configuration + std::string gsStorageRootUrl; + if (auto got = conf.find("spark.hadoop.fs.gs.storage.root.url"); got != conf.end()) { + gsStorageRootUrl = got->second; + } + if (!gsStorageRootUrl.empty()) { + std::string gcsScheme; + std::string gcsEndpoint; + + const auto sep = std::string("://"); + const auto pos = gsStorageRootUrl.find_first_of(sep); + if (pos != std::string::npos) { + gcsScheme = gsStorageRootUrl.substr(0, pos); + gcsEndpoint = gsStorageRootUrl.substr(pos + sep.length()); + } - initCache(conf); - initIOExecutor(conf); + if (!gcsEndpoint.empty() && !gcsScheme.empty()) { + veloxmemcfg->setValue("hive.gcs.scheme", gcsScheme); + veloxmemcfg->setValue("hive.gcs.endpoint", gcsEndpoint); + } + } + + // https://github.com/GoogleCloudDataproc/hadoop-connectors/blob/master/gcs/CONFIGURATION.md#authentication + std::string gsAuthType; + if (auto got = conf.find("spark.hadoop.fs.gs.auth.type"); got != conf.end()) { + gsAuthType = got->second; + } + if (gsAuthType == "SERVICE_ACCOUNT_JSON_KEYFILE") { + std::string gsAuthServiceAccountJsonKeyfile; + if (auto got = conf.find("spark.hadoop.fs.gs.auth.service.account.json.keyfile"); got != conf.end()) { + gsAuthServiceAccountJsonKeyfile = got->second; + } + + std::string gsAuthServiceAccountJson; + if (!gsAuthServiceAccountJsonKeyfile.empty()) { + auto stream = std::ifstream(gsAuthServiceAccountJsonKeyfile); + stream.exceptions(std::ios::badbit); + gsAuthServiceAccountJson = std::string(std::istreambuf_iterator(stream.rdbuf()), {}); + } else { + LOG(WARNING) << "STARTUP: conf spark.hadoop.fs.gs.auth.type is set to SERVICE_ACCOUNT_JSON_KEYFILE, " + "however conf spark.hadoop.fs.gs.auth.service.account.json.keyfile is not set"; + throw GlutenException("Conf spark.hadoop.fs.gs.auth.service.account.json.keyfile is not set"); + } -#ifdef GLUTEN_PRINT_DEBUG - printConf(conf); + if (!gsAuthServiceAccountJson.empty()) { + veloxmemcfg->setValue("hive.gcs.credentials", gsAuthServiceAccountJson); + } + } #endif - auto properties = std::make_shared(configurationValues); - velox::connector::registerConnectorFactory(std::make_shared()); + initCache(veloxcfg); + initIOExecutor(veloxcfg); + + veloxmemcfg->setValue( + velox::connector::hive::HiveConfig::kEnableFileHandleCache, + veloxcfg->get(kVeloxFileHandleCacheEnabled, kVeloxFileHandleCacheEnabledDefault) ? "true" : "false"); auto hiveConnector = velox::connector::getConnectorFactory(velox::connector::hive::HiveConnectorFactory::kHiveConnectorName) - ->newConnector(kHiveConnectorId, properties, ioExecutor_.get()); + ->newConnector(kHiveConnectorId, veloxmemcfg, ioExecutor_.get()); registerConnector(hiveConnector); - velox::parquet::registerParquetReaderFactory(velox::parquet::ParquetReaderType::NATIVE); - velox::dwrf::registerDwrfReaderFactory(); - velox::dwrf::registerOrcReaderFactory(); + // Register Velox functions registerAllFunctions(); if (!facebook::velox::isRegisteredVectorSerde()) { @@ -252,51 +265,40 @@ void VeloxBackend::init(const std::unordered_map& conf } velox::exec::Operator::registerOperator(std::make_unique()); - initUdf(conf); + initUdf(veloxcfg); + + if (veloxcfg->get(kDebugModeEnabled, false)) { + LOG(INFO) << "VeloxBackend config:" << printConfig(veloxcfg->valuesCopy()); + } } -velox::memory::MemoryAllocator* VeloxBackend::getAsyncDataCache() const { +facebook::velox::cache::AsyncDataCache* VeloxBackend::getAsyncDataCache() const { return asyncDataCache_.get(); } // JNI-or-local filesystem, for spilling-to-heap if we have extra JVM heap spaces -void VeloxBackend::initJolFilesystem(const std::unordered_map& conf) { - int64_t maxSpillFileSize = std::stol(kMaxSpillFileSizeDefault); - auto got = conf.find(kMaxSpillFileSize); - if (got != conf.end()) { - maxSpillFileSize = std::stol(got->second); - } +void VeloxBackend::initJolFilesystem(const facebook::velox::Config* conf) { + int64_t maxSpillFileSize = conf->get(kMaxSpillFileSize, kMaxSpillFileSizeDefault); + // FIXME It's known that if spill compression is disabled, the actual spill file size may // in crease beyond this limit a little (maximum 64 rows which is by default // one compression page) gluten::registerJolFileSystem(maxSpillFileSize); } -void VeloxBackend::initCache(const std::unordered_map& conf) { - auto key = conf.find(kVeloxCacheEnabled); - if (key != conf.end() && boost::algorithm::to_lower_copy(conf.at(kVeloxCacheEnabled)) == "true") { +void VeloxBackend::initCache(const facebook::velox::Config* conf) { + bool veloxCacheEnabled = conf->get(kVeloxCacheEnabled, false); + if (veloxCacheEnabled) { FLAGS_ssd_odirect = true; - if (conf.find(kVeloxSsdODirectEnabled) != conf.end() && - boost::algorithm::to_lower_copy(conf.at(kVeloxSsdODirectEnabled)) == "false") { - FLAGS_ssd_odirect = false; - } - uint64_t memCacheSize = std::stol(kVeloxMemCacheSizeDefault); - uint64_t ssdCacheSize = std::stol(kVeloxSsdCacheSizeDefault); - int32_t ssdCacheShards = std::stoi(kVeloxSsdCacheShardsDefault); - int32_t ssdCacheIOThreads = std::stoi(kVeloxSsdCacheIOThreadsDefault); - std::string ssdCachePathPrefix = kVeloxSsdCachePathDefault; - for (auto& [k, v] : conf) { - if (k == kVeloxMemCacheSize) - memCacheSize = std::stol(v); - if (k == kVeloxSsdCacheSize) - ssdCacheSize = std::stol(v); - if (k == kVeloxSsdCacheShards) - ssdCacheShards = std::stoi(v); - if (k == kVeloxSsdCachePath) - ssdCachePathPrefix = v; - if (k == kVeloxSsdCacheIOThreads) - ssdCacheIOThreads = std::stoi(v); - } + + FLAGS_ssd_odirect = conf->get(kVeloxSsdODirectEnabled, false); + + uint64_t memCacheSize = conf->get(kVeloxMemCacheSize, kVeloxMemCacheSizeDefault); + uint64_t ssdCacheSize = conf->get(kVeloxSsdCacheSize, kVeloxSsdCacheSizeDefault); + int32_t ssdCacheShards = conf->get(kVeloxSsdCacheShards, kVeloxSsdCacheShardsDefault); + int32_t ssdCacheIOThreads = conf->get(kVeloxSsdCacheIOThreads, kVeloxSsdCacheIOThreadsDefault); + std::string ssdCachePathPrefix = conf->get(kVeloxSsdCachePath, kVeloxSsdCachePathDefault); + cachePathPrefix_ = ssdCachePathPrefix; cacheFilePrefix_ = getCacheFilePrefix(); std::string ssdCachePath = ssdCachePathPrefix + "/" + cacheFilePrefix_; @@ -317,9 +319,11 @@ void VeloxBackend::initCache(const std::unordered_map& auto allocator = std::make_shared(options); if (ssdCacheSize == 0) { LOG(INFO) << "AsyncDataCache will do memory caching only as ssd cache size is 0"; - asyncDataCache_ = std::make_shared(allocator, memCacheSize, nullptr); + // TODO: this is not tracked by Spark. + asyncDataCache_ = velox::cache::AsyncDataCache::create(allocator.get()); } else { - asyncDataCache_ = std::make_shared(allocator, memCacheSize, std::move(ssd)); + // TODO: this is not tracked by Spark. + asyncDataCache_ = velox::cache::AsyncDataCache::create(allocator.get(), std::move(ssd)); } VELOX_CHECK_NOT_NULL(dynamic_cast(asyncDataCache_.get())) @@ -329,10 +333,9 @@ void VeloxBackend::initCache(const std::unordered_map& } } -void VeloxBackend::initIOExecutor(const std::unordered_map& conf) { - int32_t ioThreads = std::stoi(getConfigValue(conf, kVeloxIOThreads, kVeloxIOThreadsDefault)); - int32_t splitPreloadPerDriver = - std::stoi(getConfigValue(conf, kVeloxSplitPreloadPerDriver, kVeloxSplitPreloadPerDriverDefault)); +void VeloxBackend::initIOExecutor(const facebook::velox::Config* conf) { + int32_t ioThreads = conf->get(kVeloxIOThreads, kVeloxIOThreadsDefault); + int32_t splitPreloadPerDriver = conf->get(kVeloxSplitPreloadPerDriver, kVeloxSplitPreloadPerDriverDefault); if (ioThreads > 0) { ioExecutor_ = std::make_unique(ioThreads); FLAGS_split_preload_per_driver = splitPreloadPerDriver; @@ -344,11 +347,11 @@ void VeloxBackend::initIOExecutor(const std::unordered_map& conf) { - auto got = conf.find(kVeloxUdfLibraryPaths); - if (got != conf.end() && !got->second.empty()) { +void VeloxBackend::initUdf(const facebook::velox::Config* conf) { + auto got = conf->get(kVeloxUdfLibraryPaths, ""); + if (!got.empty()) { auto udfLoader = gluten::UdfLoader::getInstance(); - udfLoader->loadUdfLibraries(got->second); + udfLoader->loadUdfLibraries(got); udfLoader->registerUdf(); } } diff --git a/cpp/velox/compute/VeloxBackend.h b/cpp/velox/compute/VeloxBackend.h index 686862333ea0..19aa17eca6b5 100644 --- a/cpp/velox/compute/VeloxBackend.h +++ b/cpp/velox/compute/VeloxBackend.h @@ -26,6 +26,7 @@ #include "velox/common/caching/AsyncDataCache.h" #include "velox/common/memory/MemoryPool.h" +#include "velox/core/Config.h" namespace gluten { /// As a static instance in per executor, initialized at executor startup. @@ -48,7 +49,7 @@ class VeloxBackend { static VeloxBackend* get(); - facebook::velox::memory::MemoryAllocator* getAsyncDataCache() const; + facebook::velox::cache::AsyncDataCache* getAsyncDataCache() const; private: explicit VeloxBackend(const std::unordered_map& conf) { @@ -56,13 +57,11 @@ class VeloxBackend { } void init(const std::unordered_map& conf); - void initCache(const std::unordered_map& conf); - void initIOExecutor(const std::unordered_map& conf); - void initUdf(const std::unordered_map& conf); + void initCache(const facebook::velox::Config* conf); + void initIOExecutor(const facebook::velox::Config* conf); + void initUdf(const facebook::velox::Config* conf); - void initJolFilesystem(const std::unordered_map& conf); - - void printConf(const std::unordered_map& conf); + void initJolFilesystem(const facebook::velox::Config* conf); std::string getCacheFilePrefix() { return "cache." + boost::lexical_cast(boost::uuids::random_generator()()) + "."; @@ -71,8 +70,7 @@ class VeloxBackend { static std::unique_ptr instance_; // Instance of AsyncDataCache used for all large allocations. - std::shared_ptr asyncDataCache_ = - facebook::velox::memory::MemoryAllocator::createDefaultInstance(); + std::shared_ptr asyncDataCache_; std::unique_ptr ssdCacheExecutor_; std::unique_ptr ioExecutor_; diff --git a/cpp/velox/compute/VeloxExecutionCtx.cc b/cpp/velox/compute/VeloxExecutionCtx.cc deleted file mode 100644 index f8c81483830f..000000000000 --- a/cpp/velox/compute/VeloxExecutionCtx.cc +++ /dev/null @@ -1,262 +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 "VeloxExecutionCtx.h" -#include - -#include "arrow/c/bridge.h" -#include "compute/ExecutionCtx.h" -#include "compute/ResultIterator.h" -#include "compute/VeloxPlanConverter.h" -#include "config/GlutenConfig.h" -#include "operators/serializer/VeloxRowToColumnarConverter.h" -#include "shuffle/VeloxShuffleWriter.h" - -using namespace facebook; - -namespace gluten { - -namespace { - -#ifdef GLUTEN_PRINT_DEBUG -void printSessionConf(const std::unordered_map& conf) { - std::ostringstream oss; - oss << "session conf = {\n"; - for (auto& [k, v] : conf) { - oss << " {" << k << " = " << v << "}\n"; - } - oss << "}\n"; - LOG(INFO) << oss.str(); -} -#endif - -} // namespace - -VeloxExecutionCtx::VeloxExecutionCtx(const std::unordered_map& confMap) - : ExecutionCtx(confMap) {} - -void VeloxExecutionCtx::getInfoAndIds( - const std::unordered_map>& splitInfoMap, - const std::unordered_set& leafPlanNodeIds, - std::vector>& scanInfos, - std::vector& scanIds, - std::vector& streamIds) { - for (const auto& leafPlanNodeId : leafPlanNodeIds) { - auto it = splitInfoMap.find(leafPlanNodeId); - if (it == splitInfoMap.end()) { - throw std::runtime_error("Could not find leafPlanNodeId."); - } - auto splitInfo = it->second; - if (splitInfo->isStream) { - streamIds.emplace_back(leafPlanNodeId); - } else { - scanInfos.emplace_back(splitInfo); - scanIds.emplace_back(leafPlanNodeId); - } - } -} - -ResourceHandle VeloxExecutionCtx::createResultIterator( - MemoryManager* memoryManager, - const std::string& spillDir, - const std::vector>& inputs, - const std::unordered_map& sessionConf) { -#ifdef GLUTEN_PRINT_DEBUG - printSessionConf(sessionConf); -#endif - - auto veloxPool = getAggregateVeloxPool(memoryManager); - - VeloxPlanConverter veloxPlanConverter(inputs, getLeafVeloxPool(memoryManager).get(), sessionConf); - veloxPlan_ = veloxPlanConverter.toVeloxPlan(substraitPlan_); - - // Scan node can be required. - std::vector> scanInfos; - std::vector scanIds; - std::vector streamIds; - - // Separate the scan ids and stream ids, and get the scan infos. - getInfoAndIds(veloxPlanConverter.splitInfos(), veloxPlan_->leafPlanNodeIds(), scanInfos, scanIds, streamIds); - - if (scanInfos.size() == 0) { - // Source node is not required. - auto wholestageIter = std::make_unique( - veloxPool, veloxPlan_, streamIds, spillDir, sessionConf, taskInfo_); - auto resultIter = std::make_shared(std::move(wholestageIter), this); - return resultIteratorHolder_.insert(std::move(resultIter)); - } else { - auto wholestageIter = std::make_unique( - veloxPool, veloxPlan_, scanIds, scanInfos, streamIds, spillDir, sessionConf, taskInfo_); - auto resultIter = std::make_shared(std::move(wholestageIter), this); - return resultIteratorHolder_.insert(std::move(resultIter)); - } -} - -ResourceHandle VeloxExecutionCtx::addResultIterator(std::shared_ptr iterator) { - return resultIteratorHolder_.insert(std::move(iterator)); -} - -std::shared_ptr VeloxExecutionCtx::getResultIterator(ResourceHandle iterHandle) { - auto instance = resultIteratorHolder_.lookup(iterHandle); - if (!instance) { - std::string errorMessage = "invalid handle for ResultIterator " + std::to_string(iterHandle); - throw gluten::GlutenException(errorMessage); - } - return instance; -} - -void VeloxExecutionCtx::releaseResultIterator(ResourceHandle iterHandle) { - resultIteratorHolder_.erase(iterHandle); -} - -ResourceHandle VeloxExecutionCtx::createColumnar2RowConverter(MemoryManager* memoryManager) { - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return columnarToRowConverterHolder_.insert(std::make_shared(ctxVeloxPool)); -} - -std::shared_ptr VeloxExecutionCtx::getColumnar2RowConverter(ResourceHandle handle) { - return columnarToRowConverterHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseColumnar2RowConverter(ResourceHandle handle) { - columnarToRowConverterHolder_.erase(handle); -} - -ResourceHandle VeloxExecutionCtx::addBatch(std::shared_ptr batch) { - return columnarBatchHolder_.insert(std::move(batch)); -} - -std::shared_ptr VeloxExecutionCtx::getBatch(ResourceHandle handle) { - return columnarBatchHolder_.lookup(handle); -} - -ResourceHandle VeloxExecutionCtx::createOrGetEmptySchemaBatch(int32_t numRows) { - auto& lookup = emptySchemaBatchLoopUp_; - if (lookup.find(numRows) == lookup.end()) { - const std::shared_ptr& batch = gluten::createZeroColumnBatch(numRows); - lookup.emplace(numRows, addBatch(batch)); // the batch will be released after Spark task ends - } - return lookup.at(numRows); -} - -void VeloxExecutionCtx::releaseBatch(ResourceHandle handle) { - columnarBatchHolder_.erase(handle); -} - -ResourceHandle -VeloxExecutionCtx::select(MemoryManager* memoryManager, ResourceHandle handle, std::vector columnIndices) { - auto batch = columnarBatchHolder_.lookup(handle); - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - auto veloxBatch = gluten::VeloxColumnarBatch::from(ctxVeloxPool.get(), batch); - auto outputBatch = veloxBatch->select(ctxVeloxPool.get(), std::move(columnIndices)); - return columnarBatchHolder_.insert(outputBatch); -} - -ResourceHandle VeloxExecutionCtx::createRow2ColumnarConverter( - MemoryManager* memoryManager, - struct ArrowSchema* cSchema) { - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return rowToColumnarConverterHolder_.insert(std::make_shared(cSchema, ctxVeloxPool)); -} - -std::shared_ptr VeloxExecutionCtx::getRow2ColumnarConverter(ResourceHandle handle) { - return rowToColumnarConverterHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseRow2ColumnarConverter(ResourceHandle handle) { - rowToColumnarConverterHolder_.erase(handle); -} - -ResourceHandle VeloxExecutionCtx::createShuffleWriter( - int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, - MemoryManager* memoryManager) { - auto ctxPool = getLeafVeloxPool(memoryManager); - GLUTEN_ASSIGN_OR_THROW( - auto shuffle_writer, - VeloxShuffleWriter::create(numPartitions, std::move(partitionWriterCreator), std::move(options), ctxPool)); - return shuffleWriterHolder_.insert(std::move(shuffle_writer)); -} - -std::shared_ptr VeloxExecutionCtx::getShuffleWriter(ResourceHandle handle) { - return shuffleWriterHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseShuffleWriter(ResourceHandle handle) { - shuffleWriterHolder_.erase(handle); -} - -ResourceHandle VeloxExecutionCtx::createDatasource( - const std::string& filePath, - MemoryManager* memoryManager, - std::shared_ptr schema) { - auto veloxPool = getAggregateVeloxPool(memoryManager); - return datasourceHolder_.insert(std::make_shared(filePath, veloxPool, schema)); -} - -std::shared_ptr VeloxExecutionCtx::getDatasource(ResourceHandle handle) { - return datasourceHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseDatasource(ResourceHandle handle) { - datasourceHolder_.erase(handle); -} - -ResourceHandle VeloxExecutionCtx::createShuffleReader( - std::shared_ptr schema, - ReaderOptions options, - arrow::MemoryPool* pool, - MemoryManager* memoryManager) { - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return shuffleReaderHolder_.insert(std::make_shared(schema, options, pool, ctxVeloxPool)); -} - -std::shared_ptr VeloxExecutionCtx::getShuffleReader(ResourceHandle handle) { - return shuffleReaderHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseShuffleReader(ResourceHandle handle) { - shuffleReaderHolder_.erase(handle); -} - -std::unique_ptr VeloxExecutionCtx::createTempColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) { - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return std::make_unique(arrowPool, ctxVeloxPool, cSchema); -} - -ResourceHandle VeloxExecutionCtx::createColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) { - auto ctxVeloxPool = getLeafVeloxPool(memoryManager); - return columnarBatchSerializerHolder_.insert( - std::make_shared(arrowPool, ctxVeloxPool, cSchema)); -} - -std::shared_ptr VeloxExecutionCtx::getColumnarBatchSerializer(ResourceHandle handle) { - return columnarBatchSerializerHolder_.lookup(handle); -} - -void VeloxExecutionCtx::releaseColumnarBatchSerializer(ResourceHandle handle) { - columnarBatchSerializerHolder_.erase(handle); -} - -} // namespace gluten diff --git a/cpp/velox/compute/VeloxPlanConverter.cc b/cpp/velox/compute/VeloxPlanConverter.cc index f04ca6b6c078..a449b261234e 100644 --- a/cpp/velox/compute/VeloxPlanConverter.cc +++ b/cpp/velox/compute/VeloxPlanConverter.cc @@ -32,8 +32,12 @@ using namespace facebook; VeloxPlanConverter::VeloxPlanConverter( const std::vector>& inputIters, velox::memory::MemoryPool* veloxPool, - const std::unordered_map& confMap) - : inputIters_(inputIters), substraitVeloxPlanConverter_(veloxPool, confMap), pool_(veloxPool) {} + const std::unordered_map& confMap, + bool validationMode) + : inputIters_(inputIters), + validationMode_(validationMode), + substraitVeloxPlanConverter_(veloxPool, confMap, validationMode), + pool_(veloxPool) {} void VeloxPlanConverter::setInputPlanNode(const ::substrait::FetchRel& fetchRel) { if (fetchRel.has_input()) { @@ -118,19 +122,16 @@ void VeloxPlanConverter::setInputPlanNode(const ::substrait::ReadRel& sread) { if (iterIdx == -1) { return; } - if (inputIters_.size() == 0) { - throw std::runtime_error("Invalid input iterator."); - } // Get the input schema of this iterator. uint64_t colNum = 0; - std::vector> subTypeList; + std::vector veloxTypeList; if (sread.has_base_schema()) { const auto& baseSchema = sread.base_schema(); // Input names is not used. Instead, new input/output names will be created // because the ValueStreamNode in Velox does not support name change. colNum = baseSchema.names().size(); - subTypeList = SubstraitParser::parseNamedStruct(baseSchema); + veloxTypeList = SubstraitParser::parseNamedStruct(baseSchema); } std::vector outNames; @@ -140,12 +141,16 @@ void VeloxPlanConverter::setInputPlanNode(const ::substrait::ReadRel& sread) { outNames.emplace_back(colName); } - std::vector veloxTypeList; - for (auto subType : subTypeList) { - veloxTypeList.push_back(toVeloxType(subType->type)); + std::shared_ptr iterator; + if (!validationMode_) { + if (inputIters_.size() == 0) { + throw std::runtime_error("Invalid input iterator."); + } + iterator = inputIters_[iterIdx]; } + auto outputType = ROW(std::move(outNames), std::move(veloxTypeList)); - auto vectorStream = std::make_shared(pool_, std::move(inputIters_[iterIdx]), outputType); + auto vectorStream = std::make_shared(pool_, std::move(iterator), outputType); auto valuesNode = std::make_shared(nextPlanNodeId(), outputType, std::move(vectorStream)); substraitVeloxPlanConverter_.insertInputNode(iterIdx, valuesNode, planNodeId_); } diff --git a/cpp/velox/compute/VeloxPlanConverter.h b/cpp/velox/compute/VeloxPlanConverter.h index 7307f714e494..90c58774aa0d 100644 --- a/cpp/velox/compute/VeloxPlanConverter.h +++ b/cpp/velox/compute/VeloxPlanConverter.h @@ -32,7 +32,8 @@ class VeloxPlanConverter { explicit VeloxPlanConverter( const std::vector>& inputIters, facebook::velox::memory::MemoryPool* veloxPool, - const std::unordered_map& confMap); + const std::unordered_map& confMap, + bool validationMode = false); std::shared_ptr toVeloxPlan(::substrait::Plan& substraitPlan); @@ -71,6 +72,8 @@ class VeloxPlanConverter { std::vector> inputIters_; + bool validationMode_; + SubstraitToVeloxPlanConverter substraitVeloxPlanConverter_; facebook::velox::memory::MemoryPool* pool_; diff --git a/cpp/velox/compute/VeloxRuntime.cc b/cpp/velox/compute/VeloxRuntime.cc new file mode 100644 index 000000000000..9dbeceaf425a --- /dev/null +++ b/cpp/velox/compute/VeloxRuntime.cc @@ -0,0 +1,185 @@ +/* + * 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 "VeloxRuntime.h" + +#include + +#include "VeloxBackend.h" +#include "compute/ResultIterator.h" +#include "compute/Runtime.h" +#include "compute/VeloxPlanConverter.h" +#include "config/GlutenConfig.h" +#include "operators/serializer/VeloxRowToColumnarConverter.h" +#include "shuffle/VeloxShuffleReader.h" +#include "shuffle/VeloxShuffleWriter.h" +#include "utils/ConfigExtractor.h" + +using namespace facebook; + +namespace gluten { + +VeloxRuntime::VeloxRuntime(const std::unordered_map& confMap) : Runtime(confMap) {} + +void VeloxRuntime::parsePlan(const uint8_t* data, int32_t size, SparkTaskInfo taskInfo) { + taskInfo_ = taskInfo; + if (debugModeEnabled(confMap_)) { + try { + auto jsonPlan = substraitFromPbToJson("Plan", data, size); + LOG(INFO) << std::string(50, '#') << " received substrait::Plan:"; + LOG(INFO) << taskInfo_ << std::endl << jsonPlan; + } catch (const std::exception& e) { + LOG(WARNING) << "Error converting Substrait plan to JSON: " << e.what(); + } + } + + GLUTEN_CHECK(parseProtobuf(data, size, &substraitPlan_) == true, "Parse substrait plan failed"); +} + +void VeloxRuntime::getInfoAndIds( + const std::unordered_map>& splitInfoMap, + const std::unordered_set& leafPlanNodeIds, + std::vector>& scanInfos, + std::vector& scanIds, + std::vector& streamIds) { + for (const auto& leafPlanNodeId : leafPlanNodeIds) { + auto it = splitInfoMap.find(leafPlanNodeId); + if (it == splitInfoMap.end()) { + throw std::runtime_error("Could not find leafPlanNodeId."); + } + auto splitInfo = it->second; + if (splitInfo->isStream) { + streamIds.emplace_back(leafPlanNodeId); + } else { + scanInfos.emplace_back(splitInfo); + scanIds.emplace_back(leafPlanNodeId); + } + } +} + +std::string VeloxRuntime::planString(bool details, const std::unordered_map& sessionConf) { + std::vector> inputs; + auto veloxMemoryPool = gluten::defaultLeafVeloxMemoryPool(); + VeloxPlanConverter veloxPlanConverter(inputs, veloxMemoryPool.get(), sessionConf, true); + auto veloxPlan = veloxPlanConverter.toVeloxPlan(substraitPlan_); + return veloxPlan->toString(details, true); +} + +std::shared_ptr VeloxRuntime::createResultIterator( + MemoryManager* memoryManager, + const std::string& spillDir, + const std::vector>& inputs, + const std::unordered_map& sessionConf) { + if (debugModeEnabled(confMap_)) { + LOG(INFO) << "VeloxRuntime session config:" << printConfig(confMap_); + } + + VeloxPlanConverter veloxPlanConverter(inputs, getLeafVeloxPool(memoryManager).get(), sessionConf); + veloxPlan_ = veloxPlanConverter.toVeloxPlan(substraitPlan_); + + // Scan node can be required. + std::vector> scanInfos; + std::vector scanIds; + std::vector streamIds; + + // Separate the scan ids and stream ids, and get the scan infos. + getInfoAndIds(veloxPlanConverter.splitInfos(), veloxPlan_->leafPlanNodeIds(), scanInfos, scanIds, streamIds); + + auto* vmm = toVeloxMemoryManager(memoryManager); + if (scanInfos.size() == 0) { + // Source node is not required. + auto wholestageIter = std::make_unique( + vmm, veloxPlan_, streamIds, spillDir, sessionConf, taskInfo_); + auto resultIter = std::make_shared(std::move(wholestageIter), this); + return resultIter; + } else { + auto wholestageIter = std::make_unique( + vmm, veloxPlan_, scanIds, scanInfos, streamIds, spillDir, sessionConf, taskInfo_); + auto resultIter = std::make_shared(std::move(wholestageIter), this); + return resultIter; + } +} + +std::shared_ptr VeloxRuntime::createColumnar2RowConverter(MemoryManager* memoryManager) { + auto ctxVeloxPool = getLeafVeloxPool(memoryManager); + return std::make_shared(ctxVeloxPool); +} + +std::shared_ptr VeloxRuntime::createOrGetEmptySchemaBatch(int32_t numRows) { + auto& lookup = emptySchemaBatchLoopUp_; + if (lookup.find(numRows) == lookup.end()) { + const std::shared_ptr& batch = gluten::createZeroColumnBatch(numRows); + lookup.emplace(numRows, batch); // the batch will be released after Spark task ends + } + return lookup.at(numRows); +} + +std::shared_ptr VeloxRuntime::select( + MemoryManager* memoryManager, + std::shared_ptr batch, + std::vector columnIndices) { + auto ctxVeloxPool = getLeafVeloxPool(memoryManager); + auto veloxBatch = gluten::VeloxColumnarBatch::from(ctxVeloxPool.get(), batch); + auto outputBatch = veloxBatch->select(ctxVeloxPool.get(), std::move(columnIndices)); + return outputBatch; +} + +std::shared_ptr VeloxRuntime::createRow2ColumnarConverter( + MemoryManager* memoryManager, + struct ArrowSchema* cSchema) { + auto ctxVeloxPool = getLeafVeloxPool(memoryManager); + return std::make_shared(cSchema, ctxVeloxPool); +} + +std::shared_ptr VeloxRuntime::createShuffleWriter( + int numPartitions, + std::shared_ptr partitionWriterCreator, + const ShuffleWriterOptions& options, + MemoryManager* memoryManager) { + auto ctxPool = getLeafVeloxPool(memoryManager); + GLUTEN_ASSIGN_OR_THROW( + auto shuffle_writer, + VeloxShuffleWriter::create(numPartitions, std::move(partitionWriterCreator), std::move(options), ctxPool)); + return shuffle_writer; +} + +std::shared_ptr VeloxRuntime::createDatasource( + const std::string& filePath, + MemoryManager* memoryManager, + std::shared_ptr schema) { + auto veloxPool = getAggregateVeloxPool(memoryManager); + return std::make_shared(filePath, veloxPool, schema); +} + +std::shared_ptr VeloxRuntime::createShuffleReader( + std::shared_ptr schema, + ShuffleReaderOptions options, + arrow::MemoryPool* pool, + MemoryManager* memoryManager) { + auto ctxVeloxPool = getLeafVeloxPool(memoryManager); + return std::make_shared(schema, options, pool, ctxVeloxPool); +} + +std::unique_ptr VeloxRuntime::createColumnarBatchSerializer( + MemoryManager* memoryManager, + arrow::MemoryPool* arrowPool, + struct ArrowSchema* cSchema) { + auto ctxVeloxPool = getLeafVeloxPool(memoryManager); + return std::make_unique(arrowPool, ctxVeloxPool, cSchema); +} + +} // namespace gluten diff --git a/cpp/velox/compute/VeloxExecutionCtx.h b/cpp/velox/compute/VeloxRuntime.h similarity index 51% rename from cpp/velox/compute/VeloxExecutionCtx.h rename to cpp/velox/compute/VeloxRuntime.h index 2c4c2132504d..2d5d727624f2 100644 --- a/cpp/velox/compute/VeloxExecutionCtx.h +++ b/cpp/velox/compute/VeloxRuntime.h @@ -18,38 +18,38 @@ #pragma once #include "WholeStageResultIterator.h" -#include "compute/ExecutionCtx.h" +#include "compute/Runtime.h" #include "memory/VeloxMemoryManager.h" #include "operators/serializer/VeloxColumnarBatchSerializer.h" #include "operators/serializer/VeloxColumnarToRowConverter.h" #include "operators/writer/VeloxParquetDatasource.h" #include "shuffle/ShuffleReader.h" #include "shuffle/ShuffleWriter.h" -#include "shuffle/VeloxShuffleReader.h" -#include "utils/ResourceMap.h" namespace gluten { // This kind string must be same with VeloxBackend#name in java side. -inline static const std::string kVeloxExecutionCtxKind{"velox"}; +inline static const std::string kVeloxRuntimeKind{"velox"}; -class VeloxExecutionCtx final : public ExecutionCtx { +class VeloxRuntime final : public Runtime { public: - explicit VeloxExecutionCtx(const std::unordered_map& confMap); + explicit VeloxRuntime(const std::unordered_map& confMap); + + void parsePlan(const uint8_t* data, int32_t size, SparkTaskInfo taskInfo) override; static std::shared_ptr getAggregateVeloxPool(MemoryManager* memoryManager) { - if (auto veloxMemoryManager = dynamic_cast(memoryManager)) { - return veloxMemoryManager->getAggregateMemoryPool(); - } else { - GLUTEN_CHECK(false, "Should use VeloxMemoryManager here."); - } + return toVeloxMemoryManager(memoryManager)->getAggregateMemoryPool(); } static std::shared_ptr getLeafVeloxPool(MemoryManager* memoryManager) { + return toVeloxMemoryManager(memoryManager)->getLeafMemoryPool(); + } + + static VeloxMemoryManager* toVeloxMemoryManager(MemoryManager* memoryManager) { if (auto veloxMemoryManager = dynamic_cast(memoryManager)) { - return veloxMemoryManager->getLeafMemoryPool(); + return veloxMemoryManager; } else { - GLUTEN_CHECK(false, "Should use VeloxMemoryManager here."); + GLUTEN_CHECK(false, "Velox memory manager should be used for Velox runtime."); } } @@ -61,68 +61,53 @@ class VeloxExecutionCtx final : public ExecutionCtx { } // FIXME This is not thread-safe? - ResourceHandle createResultIterator( + std::shared_ptr createResultIterator( MemoryManager* memoryManager, const std::string& spillDir, const std::vector>& inputs = {}, const std::unordered_map& sessionConf = {}) override; - ResourceHandle addResultIterator(std::shared_ptr ptr) override; - std::shared_ptr getResultIterator(ResourceHandle handle) override; - void releaseResultIterator(ResourceHandle handle) override; - - ResourceHandle createColumnar2RowConverter(MemoryManager* memoryManager) override; - std::shared_ptr getColumnar2RowConverter(ResourceHandle handle) override; - void releaseColumnar2RowConverter(ResourceHandle handle) override; - - ResourceHandle addBatch(std::shared_ptr ptr) override; - std::shared_ptr getBatch(ResourceHandle handle) override; - ResourceHandle createOrGetEmptySchemaBatch(int32_t numRows) override; - void releaseBatch(ResourceHandle handle) override; - ResourceHandle select(MemoryManager* memoryManager, ResourceHandle batch, std::vector columnIndices) - override; - - ResourceHandle createRow2ColumnarConverter(MemoryManager* memoryManager, struct ArrowSchema* cSchema) override; - std::shared_ptr getRow2ColumnarConverter(ResourceHandle handle) override; - void releaseRow2ColumnarConverter(ResourceHandle handle) override; - - ResourceHandle createShuffleWriter( + + std::shared_ptr createColumnar2RowConverter(MemoryManager* memoryManager) override; + + std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) override; + + std::shared_ptr select( + MemoryManager* memoryManager, + std::shared_ptr batch, + std::vector columnIndices) override; + + std::shared_ptr createRow2ColumnarConverter( + MemoryManager* memoryManager, + struct ArrowSchema* cSchema) override; + + std::shared_ptr createShuffleWriter( int numPartitions, std::shared_ptr partitionWriterCreator, const ShuffleWriterOptions& options, MemoryManager* memoryManager) override; - std::shared_ptr getShuffleWriter(ResourceHandle handle) override; - void releaseShuffleWriter(ResourceHandle handle) override; Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) override { auto iter = static_cast(rawIter); return iter->getMetrics(exportNanos); } - ResourceHandle createDatasource( + std::shared_ptr createDatasource( const std::string& filePath, MemoryManager* memoryManager, std::shared_ptr schema) override; - std::shared_ptr getDatasource(ResourceHandle handle) override; - void releaseDatasource(ResourceHandle handle) override; - ResourceHandle createShuffleReader( + std::shared_ptr createShuffleReader( std::shared_ptr schema, - ReaderOptions options, + ShuffleReaderOptions options, arrow::MemoryPool* pool, MemoryManager* memoryManager) override; - std::shared_ptr getShuffleReader(ResourceHandle handle) override; - void releaseShuffleReader(ResourceHandle handle) override; - std::unique_ptr createTempColumnarBatchSerializer( + std::unique_ptr createColumnarBatchSerializer( MemoryManager* memoryManager, arrow::MemoryPool* arrowPool, struct ArrowSchema* cSchema) override; - ResourceHandle createColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) override; - std::shared_ptr getColumnarBatchSerializer(ResourceHandle handle) override; - void releaseColumnarBatchSerializer(ResourceHandle handle) override; + + std::string planString(bool details, const std::unordered_map& sessionConf) override; std::shared_ptr getVeloxPlan() { return veloxPlan_; @@ -136,18 +121,9 @@ class VeloxExecutionCtx final : public ExecutionCtx { std::vector& streamIds); private: - ResourceMap> columnarBatchHolder_; - ResourceMap> datasourceHolder_; - ResourceMap> columnarToRowConverterHolder_; - ResourceMap> shuffleReaderHolder_; - ResourceMap> shuffleWriterHolder_; - ResourceMap> columnarBatchSerializerHolder_; - ResourceMap> rowToColumnarConverterHolder_; - ResourceMap> resultIteratorHolder_; - std::shared_ptr veloxPlan_; - std::unordered_map emptySchemaBatchLoopUp_; + std::unordered_map> emptySchemaBatchLoopUp_; }; } // namespace gluten diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index 9541359de62b..da33d9538a17 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -16,7 +16,7 @@ */ #include "WholeStageResultIterator.h" #include "VeloxBackend.h" -#include "VeloxExecutionCtx.h" +#include "VeloxRuntime.h" #include "config/GlutenConfig.h" #include "velox/connectors/hive/HiveConfig.h" #include "velox/connectors/hive/HiveConnectorSplit.h" @@ -53,6 +53,7 @@ const std::string kSpillStartPartitionBit = "spark.gluten.sql.columnar.backend.v const std::string kSpillPartitionBits = "spark.gluten.sql.columnar.backend.velox.spillPartitionBits"; const std::string kSpillableReservationGrowthPct = "spark.gluten.sql.columnar.backend.velox.spillableReservationGrowthPct"; +const std::string kSpillCompressionKind = "spark.io.compression.codec"; const std::string kMaxPartialAggregationMemoryRatio = "spark.gluten.sql.columnar.backend.velox.maxPartialAggregationMemoryRatio"; const std::string kMaxExtendedPartialAggregationMemoryRatio = @@ -61,6 +62,9 @@ const std::string kAbandonPartialAggregationMinPct = "spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinPct"; const std::string kAbandonPartialAggregationMinRows = "spark.gluten.sql.columnar.backend.velox.abandonPartialAggregationMinRows"; +const std::string kBloomFilterExpectedNumItems = "spark.gluten.sql.columnar.backend.velox.bloomFilter.expectedNumItems"; +const std::string kBloomFilterNumBits = "spark.gluten.sql.columnar.backend.velox.bloomFilter.numBits"; +const std::string kBloomFilterMaxNumBits = "spark.gluten.sql.columnar.backend.velox.bloomFilter.maxNumBits"; // metrics const std::string kDynamicFiltersProduced = "dynamicFiltersProduced"; @@ -79,10 +83,11 @@ const std::string kHiveDefaultPartition = "__HIVE_DEFAULT_PARTITION__"; } // namespace WholeStageResultIterator::WholeStageResultIterator( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, - const std::unordered_map& confMap) - : veloxPlan_(planNode), confMap_(confMap), pool_(pool) { + const std::unordered_map& confMap, + const SparkTaskInfo& taskInfo) + : veloxPlan_(planNode), confMap_(confMap), taskInfo_(taskInfo), memoryManager_(memoryManager) { #ifdef ENABLE_HDFS updateHdfsTokens(); #endif @@ -95,10 +100,10 @@ std::shared_ptr WholeStageResultIterator::createNewVeloxQ connectorConfigs[kHiveConnectorId] = createConnectorConfig(); std::shared_ptr ctx = std::make_shared( nullptr, - getQueryContextConf(), + facebook::velox::core::QueryConfig{getQueryContextConf()}, connectorConfigs, gluten::VeloxBackend::get()->getAsyncDataCache(), - pool_, + memoryManager_->getAggregateMemoryPool(), nullptr, ""); return ctx; @@ -151,15 +156,10 @@ class ConditionalSuspendedSection { } // namespace int64_t WholeStageResultIterator::spillFixedSize(int64_t size) { - std::string poolName{pool_->root()->name() + "/" + pool_->name()}; + auto pool = memoryManager_->getAggregateMemoryPool(); + std::string poolName{pool->root()->name() + "/" + pool->name()}; std::string logPrefix{"Spill[" + poolName + "]: "}; - DLOG(INFO) << logPrefix << "Trying to reclaim " << size << " bytes of data..."; - DLOG(INFO) << logPrefix << "Pool has reserved " << pool_->currentBytes() << "/" << pool_->root()->reservedBytes() - << "/" << pool_->root()->capacity() << "/" << pool_->root()->maxCapacity() << " bytes."; - DLOG(INFO) << logPrefix << "Shrinking..."; - int64_t shrunken = pool_->shrinkManaged(pool_.get(), size); - DLOG(INFO) << logPrefix << shrunken << " bytes released from shrinking."; - + int64_t shrunken = memoryManager_->shrink(size); // todo return the actual spilled size? if (spillStrategy_ == "auto") { int64_t remaining = size - shrunken; @@ -173,16 +173,18 @@ int64_t WholeStageResultIterator::spillFixedSize(int64_t size) { }); // suspend the driver when we are on it ConditionalSuspendedSection noCancel(thisDriver, thisDriver != nullptr); - uint64_t spilledOut = pool_->reclaim(remaining); + velox::exec::MemoryReclaimer::Stats status; + auto* mm = memoryManager_->getMemoryManager(); + uint64_t spilledOut = mm->arbitrator()->shrinkMemory({pool}, remaining); // this conducts spilling LOG(INFO) << logPrefix << "Successfully spilled out " << spilledOut << " bytes."; uint64_t total = shrunken + spilledOut; - DLOG(INFO) << logPrefix << "Successfully reclaimed total " << total << " bytes."; + VLOG(2) << logPrefix << "Successfully reclaimed total " << total << " bytes."; return total; } else { LOG(WARNING) << "Spill-to-disk was disabled since " << kSpillStrategy << " was not configured."; } - DLOG(INFO) << logPrefix << "Successfully reclaimed total " << shrunken << " bytes."; + VLOG(2) << logPrefix << "Successfully reclaimed total " << shrunken << " bytes."; return shrunken; } @@ -209,6 +211,14 @@ void WholeStageResultIterator::collectMetrics() { return; } + if (debugModeEnabled(confMap_)) { + auto planWithStats = velox::exec::printPlanWithStats(*veloxPlan_.get(), task_->taskStats(), true); + std::ostringstream oss; + oss << "Native Plan with stats for: " << taskInfo_; + oss << "\n" << planWithStats << std::endl; + LOG(INFO) << oss.str(); + } + auto planStats = velox::exec::toPlanStats(task_->taskStats()); // Calculate the total number of metrics. int statsNum = 0; @@ -310,12 +320,14 @@ std::unordered_map WholeStageResultIterator::getQueryC try { // To align with Spark's behavior, set casting to int to be truncating. configs[velox::core::QueryConfig::kCastToIntByTruncate] = std::to_string(true); - // To align with Spark's behavior, allow decimal in casting string to int. - configs[velox::core::QueryConfig::kCastIntAllowDecimal] = std::to_string(true); + // To align with Spark's behavior, unset to support non-ISO8601 standard strings. + configs[velox::core::QueryConfig::kCastStringToDateIsIso8601] = std::to_string(false); auto defaultTimezone = getConfigValue(confMap_, kDefaultSessionTimezone, ""); configs[velox::core::QueryConfig::kSessionTimezone] = getConfigValue(confMap_, kSessionTimezone, defaultTimezone); // Adjust timestamp according to the above configured session timezone. configs[velox::core::QueryConfig::kAdjustTimestampToTimezone] = std::to_string(true); + // Align Velox size function with Spark. + configs[velox::core::QueryConfig::kSparkLegacySizeOfNull] = getConfigValue(confMap_, kLegacySize, "true"); { // partial aggregation memory config @@ -331,7 +343,7 @@ std::unordered_map WholeStageResultIterator::getQueryC configs[velox::core::QueryConfig::kAbandonPartialAggregationMinPct] = getConfigValue(confMap_, kAbandonPartialAggregationMinPct, "90"); configs[velox::core::QueryConfig::kAbandonPartialAggregationMinRows] = - getConfigValue(confMap_, kAbandonPartialAggregationMinRows, "10000"); + getConfigValue(confMap_, kAbandonPartialAggregationMinRows, "100000"); } // Spill configs if (spillStrategy_ == "none") { @@ -356,9 +368,16 @@ std::unordered_map WholeStageResultIterator::getQueryC getConfigValue(confMap_, kMinSpillRunSize, std::to_string(256 << 20)); configs[velox::core::QueryConfig::kSpillStartPartitionBit] = getConfigValue(confMap_, kSpillStartPartitionBit, "29"); - configs[velox::core::QueryConfig::kSpillPartitionBits] = getConfigValue(confMap_, kSpillPartitionBits, "2"); + configs[velox::core::QueryConfig::kJoinSpillPartitionBits] = getConfigValue(confMap_, kSpillPartitionBits, "2"); configs[velox::core::QueryConfig::kSpillableReservationGrowthPct] = getConfigValue(confMap_, kSpillableReservationGrowthPct, "25"); + configs[velox::core::QueryConfig::kSpillCompressionKind] = getConfigValue(confMap_, kSpillCompressionKind, "lz4"); + configs[velox::core::QueryConfig::kSparkBloomFilterExpectedNumItems] = + getConfigValue(confMap_, kBloomFilterExpectedNumItems, "1000000"); + configs[velox::core::QueryConfig::kSparkBloomFilterNumBits] = + getConfigValue(confMap_, kBloomFilterNumBits, "8388608"); + configs[velox::core::QueryConfig::kSparkBloomFilterMaxNumBits] = + getConfigValue(confMap_, kBloomFilterMaxNumBits, "4194304"); } catch (const std::invalid_argument& err) { std::string errDetails = err.what(); throw std::runtime_error("Invalid conf arg: " + errDetails); @@ -392,15 +411,15 @@ std::shared_ptr WholeStageResultIterator::createConnectorConfig() } WholeStageResultIteratorFirstStage::WholeStageResultIteratorFirstStage( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, const std::vector& scanNodeIds, const std::vector>& scanInfos, const std::vector& streamIds, const std::string spillDir, const std::unordered_map& confMap, - const SparkTaskInfo taskInfo) - : WholeStageResultIterator(pool, planNode, confMap), + const SparkTaskInfo& taskInfo) + : WholeStageResultIterator(memoryManager, planNode, confMap, taskInfo), scanNodeIds_(scanNodeIds), scanInfos_(scanInfos), streamIds_(streamIds) { @@ -446,10 +465,7 @@ WholeStageResultIteratorFirstStage::WholeStageResultIteratorFirstStage( std::shared_ptr queryCtx = createNewVeloxQueryCtx(); task_ = velox::exec::Task::create( - fmt::format("Gluten stage-{} task-{}", taskInfo.stageId, taskInfo.taskId), - std::move(planFragment), - 0, - std::move(queryCtx)); + fmt::format("Gluten {}", taskInfo_.toString()), std::move(planFragment), 0, std::move(queryCtx)); if (!task_->supportsSingleThreadedExecution()) { throw std::runtime_error("Task doesn't support single thread execution: " + planNode->toString()); @@ -478,7 +494,7 @@ void WholeStageResultIteratorFirstStage::constructPartitionColumns( for (const auto& partitionColumn : map) { auto key = partitionColumn.first; const auto value = partitionColumn.second; - if (!folly::to(getConfigValue(confMap_, kCaseSensitive, "true"))) { + if (!folly::to(getConfigValue(confMap_, kCaseSensitive, "false"))) { folly::toLowerAscii(key); } if (value == kHiveDefaultPartition) { @@ -490,22 +506,19 @@ void WholeStageResultIteratorFirstStage::constructPartitionColumns( } WholeStageResultIteratorMiddleStage::WholeStageResultIteratorMiddleStage( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, const std::vector& streamIds, const std::string spillDir, const std::unordered_map& confMap, - const SparkTaskInfo taskInfo) - : WholeStageResultIterator(pool, planNode, confMap), streamIds_(streamIds) { + const SparkTaskInfo& taskInfo) + : WholeStageResultIterator(memoryManager, planNode, confMap, taskInfo), streamIds_(streamIds) { std::unordered_set emptySet; velox::core::PlanFragment planFragment{planNode, velox::core::ExecutionStrategy::kUngrouped, 1, emptySet}; std::shared_ptr queryCtx = createNewVeloxQueryCtx(); task_ = velox::exec::Task::create( - fmt::format("Gluten stage-{} task-{}", taskInfo.stageId, taskInfo.taskId), - std::move(planFragment), - 0, - std::move(queryCtx)); + fmt::format("Gluten {}", taskInfo_.toString()), std::move(planFragment), 0, std::move(queryCtx)); if (!task_->supportsSingleThreadedExecution()) { throw std::runtime_error("Task doesn't support single thread execution: " + planNode->toString()); diff --git a/cpp/velox/compute/WholeStageResultIterator.h b/cpp/velox/compute/WholeStageResultIterator.h index fee25a87f999..4e1f6ac3c594 100644 --- a/cpp/velox/compute/WholeStageResultIterator.h +++ b/cpp/velox/compute/WholeStageResultIterator.h @@ -16,7 +16,7 @@ */ #pragma once -#include "compute/ExecutionCtx.h" +#include "compute/Runtime.h" #include "memory/ColumnarBatchIterator.h" #include "memory/VeloxColumnarBatch.h" #include "substrait/SubstraitToVeloxPlan.h" @@ -34,9 +34,10 @@ namespace gluten { class WholeStageResultIterator : public ColumnarBatchIterator { public: WholeStageResultIterator( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, - const std::unordered_map& confMap); + const std::unordered_map& confMap, + const SparkTaskInfo& taskInfo); virtual ~WholeStageResultIterator() { if (task_ != nullptr && task_->isRunning()) { @@ -69,6 +70,8 @@ class WholeStageResultIterator : public ColumnarBatchIterator { /// A map of custom configs. std::unordered_map confMap_; + const SparkTaskInfo taskInfo_; + private: /// Get the Spark confs to Velox query context. std::unordered_map getQueryContextConf(); @@ -93,7 +96,7 @@ class WholeStageResultIterator : public ColumnarBatchIterator { const std::unordered_map& runtimeStats, const std::string& metricId); - std::shared_ptr pool_; + VeloxMemoryManager* memoryManager_; // spill std::string spillStrategy_; @@ -110,14 +113,14 @@ class WholeStageResultIterator : public ColumnarBatchIterator { class WholeStageResultIteratorFirstStage final : public WholeStageResultIterator { public: WholeStageResultIteratorFirstStage( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, const std::vector& scanNodeIds, const std::vector>& scanInfos, const std::vector& streamIds, const std::string spillDir, const std::unordered_map& confMap, - const SparkTaskInfo taskInfo); + const SparkTaskInfo& taskInfo); private: std::vector scanNodeIds_; @@ -134,12 +137,12 @@ class WholeStageResultIteratorFirstStage final : public WholeStageResultIterator class WholeStageResultIteratorMiddleStage final : public WholeStageResultIterator { public: WholeStageResultIteratorMiddleStage( - std::shared_ptr pool, + VeloxMemoryManager* memoryManager, const std::shared_ptr& planNode, const std::vector& streamIds, const std::string spillDir, const std::unordered_map& confMap, - const SparkTaskInfo taskInfo); + const SparkTaskInfo& taskInfo); private: bool noMoreSplits_ = false; diff --git a/cpp/velox/jni/VeloxJniWrapper.cc b/cpp/velox/jni/VeloxJniWrapper.cc index 762462ef5cf4..8f1427baa95a 100644 --- a/cpp/velox/jni/VeloxJniWrapper.cc +++ b/cpp/velox/jni/VeloxJniWrapper.cc @@ -22,7 +22,7 @@ #include #include "JniUdf.h" #include "compute/VeloxBackend.h" -#include "compute/VeloxExecutionCtx.h" +#include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" #include "jni/JniError.h" #include "jni/JniFileSystem.h" @@ -34,8 +34,8 @@ using namespace facebook; namespace { -gluten::ExecutionCtx* veloxExecutionCtxFactory(const std::unordered_map& sessionConf) { - return new gluten::VeloxExecutionCtx(sessionConf); +gluten::Runtime* veloxRuntimeFactory(const std::unordered_map& sessionConf) { + return new gluten::VeloxRuntime(sessionConf); } } // namespace @@ -49,9 +49,6 @@ jint JNI_OnLoad(JavaVM* vm, void*) { return JNI_ERR; } - // logging - google::InitGoogleLogging("gluten"); - FLAGS_logtostderr = true; gluten::getJniCommonState()->ensureInitialized(env); gluten::getJniErrorState()->ensureInitialized(env); gluten::initVeloxJniFileSystem(env); @@ -78,7 +75,7 @@ JNIEXPORT void JNICALL Java_io_glutenproject_init_NativeBackendInitializer_initi jbyteArray conf) { JNI_METHOD_START auto sparkConf = gluten::parseConfMap(env, conf); - gluten::ExecutionCtx::registerFactory(gluten::kVeloxExecutionCtxKind, veloxExecutionCtxFactory); + gluten::Runtime::registerFactory(gluten::kVeloxRuntimeKind, veloxRuntimeFactory); gluten::VeloxBackend::create(sparkConf); JNI_METHOD_END() } diff --git a/cpp/velox/memory/VeloxColumnarBatch.cc b/cpp/velox/memory/VeloxColumnarBatch.cc index c6b207e12563..991a4e9b1a0c 100644 --- a/cpp/velox/memory/VeloxColumnarBatch.cc +++ b/cpp/velox/memory/VeloxColumnarBatch.cc @@ -15,7 +15,7 @@ * limitations under the License. */ #include "VeloxColumnarBatch.h" -#include "compute/VeloxExecutionCtx.h" +#include "compute/VeloxRuntime.h" #include "velox/row/UnsafeRowFast.h" #include "velox/type/Type.h" #include "velox/vector/FlatVector.h" @@ -146,6 +146,7 @@ std::pair VeloxColumnarBatch::getRowBytes(int32_t rowId) const { auto fast = std::make_unique(rowVector_); auto size = fast->rowSize(rowId); char* rowBytes = new char[size]; + std::memset(rowBytes, 0, size); fast->serialize(0, rowBytes); return std::make_pair(rowBytes, size); } diff --git a/cpp/velox/memory/VeloxMemoryManager.cc b/cpp/velox/memory/VeloxMemoryManager.cc index ada0491af82a..1c78283f2830 100644 --- a/cpp/velox/memory/VeloxMemoryManager.cc +++ b/cpp/velox/memory/VeloxMemoryManager.cc @@ -16,6 +16,9 @@ */ #include "VeloxMemoryManager.h" +#include "velox/common/memory/MallocAllocator.h" +#include "velox/common/memory/MemoryPool.h" +#include "velox/exec/MemoryReclaimer.h" #include "memory/ArrowMemoryPool.h" #include "utils/exception.h" @@ -28,73 +31,25 @@ using namespace facebook; // gluten allocator is only used to do allocation-reporting to Spark in mmap case // This allocator only hook `allocateBytes` and `freeBytes`, we can not ensure this behavior is safe enough, // so, only use this allocator when build with GLUTEN_ENABLE_HBM. -class VeloxMemoryAllocator final : public velox::memory::MemoryAllocator { +class VeloxMemoryAllocator final : public velox::memory::MallocAllocator { public: - VeloxMemoryAllocator(gluten::MemoryAllocator* glutenAlloc, velox::memory::MemoryAllocator* veloxAlloc) - : glutenAlloc_(glutenAlloc), veloxAlloc_(veloxAlloc) {} + VeloxMemoryAllocator(gluten::MemoryAllocator* glutenAlloc) + : MallocAllocator(velox::memory::kMaxMemory), glutenAlloc_(glutenAlloc) {} - Kind kind() const override { - return veloxAlloc_->kind(); - } - - bool allocateNonContiguous( - velox::memory::MachinePageCount numPages, - velox::memory::Allocation& out, - ReservationCallback reservationCB, - velox::memory::MachinePageCount minSizeClass) override { - return veloxAlloc_->allocateNonContiguous(numPages, out, reservationCB, minSizeClass); - } - - int64_t freeNonContiguous(velox::memory::Allocation& allocation) override { - int64_t freedBytes = veloxAlloc_->freeNonContiguous(allocation); - return freedBytes; - } - - bool allocateContiguous( - velox::memory::MachinePageCount numPages, - velox::memory::Allocation* collateral, - velox::memory::ContiguousAllocation& allocation, - ReservationCallback reservationCB) override { - return veloxAlloc_->allocateContiguous(numPages, collateral, allocation, reservationCB); - } - - void freeContiguous(velox::memory::ContiguousAllocation& allocation) override { - veloxAlloc_->freeContiguous(allocation); - } - - void* allocateBytes(uint64_t bytes, uint16_t alignment) override { + protected: + void* allocateBytesWithoutRetry(uint64_t bytes, uint16_t alignment) override { void* out; - VELOX_CHECK(glutenAlloc_->allocateAligned(alignment, bytes, &out)) + VELOX_CHECK(glutenAlloc_->allocateAligned(alignment, bytes, &out), "Issue allocating bytes"); return out; } + public: void freeBytes(void* p, uint64_t size) noexcept override { VELOX_CHECK(glutenAlloc_->free(p, size)); } - bool checkConsistency() const override { - return veloxAlloc_->checkConsistency(); - } - - velox::memory::MachinePageCount numAllocated() const override { - return veloxAlloc_->numAllocated(); - } - - velox::memory::MachinePageCount numMapped() const override { - return veloxAlloc_->numMapped(); - } - - std::string toString() const override { - return veloxAlloc_->toString(); - } - - size_t capacity() const override { - return veloxAlloc_->capacity(); - } - private: gluten::MemoryAllocator* glutenAlloc_; - velox::memory::MemoryAllocator* veloxAlloc_; }; /// We assume in a single Spark task. No thread-safety should be guaranteed. @@ -103,22 +58,33 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { ListenableArbitrator(const Config& config, AllocationListener* listener) : MemoryArbitrator(config), listener_(listener) {} + std::string kind() const override { + return kind_; + } + void reserveMemory(velox::memory::MemoryPool* pool, uint64_t) override { - growPool(pool, initMemoryPoolCapacity_); + std::lock_guard l(mutex_); + growPoolLocked(pool, memoryPoolInitCapacity_); } - uint64_t releaseMemory(velox::memory::MemoryPool* pool, uint64_t bytes) override { - uint64_t freeBytes = pool->shrink(bytes); - listener_->allocationChanged(-freeBytes); - if (bytes == 0 && pool->capacity() != 0) { - // So far only MemoryManager::dropPool() calls with 0 bytes. Let's assert the pool - // gives all capacity back to Spark - // - // We are likely in destructor, do not throw. INFO log is fine since we have leak checks from Spark's memory - // manager - LOG(INFO) << "Memory pool " << pool->name() << " not completely shrunken when Memory::dropPool() is called"; - } - return freeBytes; + void releaseMemory(velox::memory::MemoryPool* pool) override { + std::lock_guard l(mutex_); + releaseMemoryLocked(pool); + } + + uint64_t shrinkMemory(const std::vector>& pools, uint64_t targetBytes) + override { + facebook::velox::exec::MemoryReclaimer::Stats status; + GLUTEN_CHECK(pools.size() == 1, "Should shrink a single pool at a time"); + std::lock_guard l(mutex_); // FIXME: Do we have recursive locking for this mutex? + auto pool = pools.at(0); + const uint64_t oldCapacity = pool->capacity(); + uint64_t spilledOut = pool->reclaim(targetBytes, status); // ignore the output + uint64_t shrunken = pool->shrink(0); + const uint64_t newCapacity = pool->capacity(); + uint64_t total = oldCapacity - newCapacity; + listener_->allocationChanged(-total); + return total; } bool growMemory( @@ -128,7 +94,10 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { GLUTEN_CHECK(candidatePools.size() == 1, "ListenableArbitrator should only be used within a single root pool"); auto candidate = candidatePools.back(); GLUTEN_CHECK(pool->root() == candidate.get(), "Illegal state in ListenableArbitrator"); - growPool(pool, targetBytes); + { + std::lock_guard l(mutex_); + growPoolLocked(pool, targetBytes); + } return true; } @@ -138,59 +107,50 @@ class ListenableArbitrator : public velox::memory::MemoryArbitrator { } std::string toString() const override { - return fmt::format( - "ARBITRATOR[{}] CAPACITY {} {}", kindString(kind_), velox::succinctBytes(capacity_), stats().toString()); + return fmt::format("ARBITRATOR[{}] CAPACITY {} {}", kind_, velox::succinctBytes(capacity_), stats().toString()); } private: - void growPool(velox::memory::MemoryPool* pool, uint64_t bytes) { + void growPoolLocked(velox::memory::MemoryPool* pool, uint64_t bytes) { listener_->allocationChanged(bytes); pool->grow(bytes); } - void abort(velox::memory::MemoryPool* pool) { - try { - pool->abort(); - } catch (const std::exception& e) { - LOG(WARNING) << "Failed to abort memory pool " << pool->toString(); - } - // NOTE: no matter memory pool abort throws or not, it should have been marked - // as aborted to prevent any new memory arbitration triggered from the aborted - // memory pool. - VELOX_CHECK(pool->aborted()); + void releaseMemoryLocked(velox::memory::MemoryPool* pool) { + uint64_t freeBytes = pool->shrink(0); + listener_->allocationChanged(-freeBytes); } gluten::AllocationListener* listener_; + std::recursive_mutex mutex_; + inline static std::string kind_ = "GLUTEN"; }; -velox::memory::IMemoryManager::Options VeloxMemoryManager::getOptions( - std::shared_ptr allocator) const { - auto veloxAlloc = velox::memory::MemoryAllocator::getInstance(); - -#ifdef GLUTEN_ENABLE_HBM - wrappedAlloc_ = std::make_unique(allocator.get(), veloxAlloc); - veloxAlloc = wrappedAlloc_.get(); -#endif +class ArbitratorFactoryRegister { + public: + explicit ArbitratorFactoryRegister(gluten::AllocationListener* listener) : listener_(listener) { + static std::atomic_uint32_t id{0UL}; + kind_ = "GLUTEN_ARBITRATOR_FACTORY_" + std::to_string(id++); + velox::memory::MemoryArbitrator::registerFactory( + kind_, + [this]( + const velox::memory::MemoryArbitrator::Config& config) -> std::unique_ptr { + return std::make_unique(config, listener_); + }); + } - velox::memory::MemoryArbitrator::Config arbitratorConfig{ - velox::memory::MemoryArbitrator::Kind::kNoOp, // do not use shared arbitrator as it will mess up the thread - // contexts (one Spark task reclaims memory from another) - velox::memory::kMaxMemory, // the 2nd capacity - 0, // initMemoryPoolCapacity - 32 << 20, // minMemoryPoolCapacityTransferSize - true}; + virtual ~ArbitratorFactoryRegister() { + velox::memory::MemoryArbitrator::unregisterFactory(kind_); + } - velox::memory::IMemoryManager::Options mmOptions{ - velox::memory::MemoryAllocator::kMaxAlignment, - velox::memory::kMaxMemory, // the 1st capacity, Velox requires for a couple of different capacity numbers - false, // leak check - false, // debug - veloxAlloc, - [=]() { return std::make_unique(arbitratorConfig, listener_.get()); }, - }; + const std::string& getKind() const { + return kind_; + } - return mmOptions; -} + private: + std::string kind_; + gluten::AllocationListener* listener_; +}; VeloxMemoryManager::VeloxMemoryManager( const std::string& name, @@ -200,8 +160,30 @@ VeloxMemoryManager::VeloxMemoryManager( glutenAlloc_ = std::make_unique(allocator.get(), listener_.get()); arrowPool_ = std::make_unique(glutenAlloc_.get()); - auto options = getOptions(allocator); - veloxMemoryManager_ = std::make_unique(options); + auto veloxAlloc = velox::memory::MemoryAllocator::getInstance(); + +#ifdef GLUTEN_ENABLE_HBM + wrappedAlloc_ = std::make_unique(allocator.get(), veloxAlloc); + veloxAlloc = wrappedAlloc_.get(); +#endif + + ArbitratorFactoryRegister afr(listener_.get()); + velox::memory::MemoryManagerOptions mmOptions{ + velox::memory::MemoryAllocator::kMaxAlignment, + velox::memory::kMaxMemory, + velox::memory::kMaxMemory, + true, // memory usage tracking + true, // leak check + false, // debug +#ifdef GLUTEN_ENABLE_HBM + wrappedAlloc.get(), +#else + veloxAlloc, +#endif + afr.getKind(), + 0, + 32 << 20}; + veloxMemoryManager_ = std::make_unique(mmOptions); veloxAggregatePool_ = veloxMemoryManager_->addRootPool( name_ + "_root", @@ -212,53 +194,61 @@ VeloxMemoryManager::VeloxMemoryManager( } namespace { -MemoryUsageStats collectVeloxMemoryPoolUsageStats(const velox::memory::MemoryPool* pool) { +MemoryUsageStats collectMemoryUsageStatsInternal(const velox::memory::MemoryPool* pool) { MemoryUsageStats stats; stats.set_current(pool->currentBytes()); stats.set_peak(pool->peakBytes()); // walk down root and all children pool->visitChildren([&](velox::memory::MemoryPool* pool) -> bool { - stats.mutable_children()->emplace(pool->name(), collectVeloxMemoryPoolUsageStats(pool)); + stats.mutable_children()->emplace(pool->name(), collectMemoryUsageStatsInternal(pool)); return true; }); return stats; } -MemoryUsageStats collectArrowMemoryPoolUsageStats(const arrow::MemoryPool* pool) { - MemoryUsageStats stats; - stats.set_current(pool->bytes_allocated()); - stats.set_peak(-1LL); // we don't know about peak - return stats; +int64_t shrinkVeloxMemoryPool(velox::memory::MemoryManager* mm, velox::memory::MemoryPool* pool, int64_t size) { + std::string poolName{pool->root()->name() + "/" + pool->name()}; + std::string logPrefix{"Shrink[" + poolName + "]: "}; + VLOG(2) << logPrefix << "Trying to shrink " << size << " bytes of data..."; + VLOG(2) << logPrefix << "Pool has reserved " << pool->currentBytes() << "/" << pool->root()->reservedBytes() << "/" + << pool->root()->capacity() << "/" << pool->root()->maxCapacity() << " bytes."; + VLOG(2) << logPrefix << "Shrinking..."; + const uint64_t oldCapacity = pool->capacity(); + mm->arbitrator()->releaseMemory(pool); + const uint64_t newCapacity = pool->capacity(); + int64_t shrunken = oldCapacity - newCapacity; + VLOG(2) << logPrefix << shrunken << " bytes released from shrinking."; + return shrunken; } } // namespace const MemoryUsageStats VeloxMemoryManager::collectMemoryUsageStats() const { - const MemoryUsageStats& veloxPoolStats = collectVeloxMemoryPoolUsageStats(veloxAggregatePool_.get()); - const MemoryUsageStats& arrowPoolStats = collectArrowMemoryPoolUsageStats(arrowPool_.get()); - MemoryUsageStats stats; - stats.set_current(veloxPoolStats.current() + arrowPoolStats.current()); - stats.set_peak(-1LL); // we don't know about peak - stats.mutable_children()->emplace("velox", std::move(veloxPoolStats)); - stats.mutable_children()->emplace("arrow", std::move(arrowPoolStats)); - return stats; + return collectMemoryUsageStatsInternal(veloxAggregatePool_.get()); +} + +const int64_t VeloxMemoryManager::shrink(int64_t size) { + return shrinkVeloxMemoryPool(veloxMemoryManager_.get(), veloxAggregatePool_.get(), size); } namespace { -int64_t shrinkVeloxMemoryPool(velox::memory::MemoryPool* pool, int64_t size) { - std::string poolName{pool->root()->name() + "/" + pool->name()}; - std::string logPrefix{"Shrink[" + poolName + "]: "}; - DLOG(INFO) << logPrefix << "Trying to shrink " << size << " bytes of data..."; - DLOG(INFO) << logPrefix << "Pool has reserved " << pool->currentBytes() << "/" << pool->root()->reservedBytes() << "/" - << pool->root()->capacity() << "/" << pool->root()->maxCapacity() << " bytes."; - DLOG(INFO) << logPrefix << "Shrinking..."; - int64_t shrunken = pool->shrinkManaged(pool, size); - DLOG(INFO) << logPrefix << shrunken << " bytes released from shrinking."; - return shrunken; +void holdInternal( + std::vector>& heldVeloxPools, + const velox::memory::MemoryPool* pool) { + pool->visitChildren([&](velox::memory::MemoryPool* child) -> bool { + auto shared = child->shared_from_this(); + heldVeloxPools.push_back(shared); + holdInternal(heldVeloxPools, child); + return true; + }); } } // namespace -const int64_t VeloxMemoryManager::shrink(int64_t size) { - return shrinkVeloxMemoryPool(veloxAggregatePool_.get(), size); +void VeloxMemoryManager::hold() { + holdInternal(heldVeloxPools_, veloxAggregatePool_.get()); +} + +velox::memory::MemoryManager* getDefaultVeloxMemoryManager() { + return &(facebook::velox::memory::defaultMemoryManager()); } } // namespace gluten diff --git a/cpp/velox/memory/VeloxMemoryManager.h b/cpp/velox/memory/VeloxMemoryManager.h index 4eb58e4b2d50..875fb1847274 100644 --- a/cpp/velox/memory/VeloxMemoryManager.h +++ b/cpp/velox/memory/VeloxMemoryManager.h @@ -45,6 +45,10 @@ class VeloxMemoryManager final : public MemoryManager { return veloxLeafPool_; } + facebook::velox::memory::MemoryManager* getMemoryManager() const { + return veloxMemoryManager_.get(); + } + arrow::MemoryPool* getArrowMemoryPool() override { return arrowPool_.get(); } @@ -53,9 +57,9 @@ class VeloxMemoryManager final : public MemoryManager { const int64_t shrink(int64_t size) override; - private: - facebook::velox::memory::IMemoryManager::Options getOptions(std::shared_ptr allocator) const; + void hold() override; + private: std::string name_; #ifdef GLUTEN_ENABLE_HBM @@ -70,6 +74,7 @@ class VeloxMemoryManager final : public MemoryManager { std::unique_ptr veloxMemoryManager_; std::shared_ptr veloxAggregatePool_; std::shared_ptr veloxLeafPool_; + std::vector> heldVeloxPools_; }; /// Not tracked by Spark and should only used in test or validation. diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.cc b/cpp/velox/operators/functions/RegistrationAllFunctions.cc index 912879cb436d..83923be5f802 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.cc +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.cc @@ -14,14 +14,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -#include "RegistrationAllFunctions.h" -#include "Arithmetic.h" -#include "RowConstructor.h" +#include "operators/functions/RegistrationAllFunctions.h" +#include "operators/functions/Arithmetic.h" +#include "operators/functions/RowConstructorWithNull.h" +#include "operators/functions/RowFunctionWithNull.h" +#include "velox/expression/SpecialFormRegistry.h" +#include "velox/expression/VectorFunction.h" #include "velox/functions/lib/RegistrationHelpers.h" #include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" #include "velox/functions/prestosql/registration/RegistrationFunctions.h" #include "velox/functions/prestosql/window/WindowFunctionsRegistration.h" +#include "velox/functions/sparksql/Hash.h" #include "velox/functions/sparksql/Register.h" #include "velox/functions/sparksql/aggregates/Register.h" #include "velox/functions/sparksql/window/WindowFunctionsRegistration.h" @@ -29,27 +33,48 @@ using namespace facebook; namespace gluten { - +namespace { void registerFunctionOverwrite() { - facebook::velox::functions::registerUnaryNumeric({"round"}); - facebook::velox::registerFunction({"round"}); - facebook::velox::registerFunction({"round"}); - facebook::velox::registerFunction({"round"}); - facebook::velox::registerFunction({"round"}); - facebook::velox::registerFunction({"round"}); - facebook::velox::registerFunction({"round"}); + velox::exec::registerStatefulVectorFunction( + "murmur3hash", + velox::functions::sparksql::hashWithSeedSignatures(), + velox::functions::sparksql::makeHashWithSeed); + + velox::exec::registerStatefulVectorFunction( + "xxhash64", + velox::functions::sparksql::xxhash64WithSeedSignatures(), + velox::functions::sparksql::makeXxHash64WithSeed); + + velox::functions::registerUnaryNumeric({"round"}); + velox::registerFunction({"round"}); + velox::registerFunction({"round"}); + velox::registerFunction({"round"}); + velox::registerFunction({"round"}); + velox::registerFunction({"round"}); + velox::registerFunction({"round"}); + + velox::exec::registerVectorFunction( + "row_constructor_with_null", + std::vector>{}, + std::make_unique()); + velox::exec::registerFunctionCallToSpecialForm( + RowConstructorWithNullCallToSpecialForm::kRowConstructorWithNull, + std::make_unique()); } +} // namespace void registerAllFunctions() { // The registration order matters. Spark sql functions are registered after // presto sql functions to overwrite the registration for same named functions. velox::functions::prestosql::registerAllScalarFunctions(); velox::functions::sparksql::registerFunctions(""); - // registerCustomFunctions(); - velox::aggregate::prestosql::registerAllAggregateFunctions(); - velox::functions::aggregate::sparksql::registerAggregateFunctions(""); + velox::aggregate::prestosql::registerAllAggregateFunctions( + "", true /*registerCompanionFunctions*/, true /*overwrite*/); + velox::functions::aggregate::sparksql::registerAggregateFunctions( + "", true /*registerCompanionFunctions*/, true /*overwrite*/); velox::window::prestosql::registerAllWindowFunctions(); velox::functions::window::sparksql::registerWindowFunctions(""); + // Using function overwrite to handle function names mismatch between Spark and Velox. registerFunctionOverwrite(); } diff --git a/cpp/velox/operators/functions/RegistrationAllFunctions.h b/cpp/velox/operators/functions/RegistrationAllFunctions.h index f3279465e2fe..9ba754d14fcb 100644 --- a/cpp/velox/operators/functions/RegistrationAllFunctions.h +++ b/cpp/velox/operators/functions/RegistrationAllFunctions.h @@ -20,6 +20,5 @@ namespace gluten { void registerAllFunctions(); -void registerFunctionOverwrite(); } // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.cc b/cpp/velox/operators/functions/RowConstructorWithNull.cc new file mode 100644 index 000000000000..411dd478e07e --- /dev/null +++ b/cpp/velox/operators/functions/RowConstructorWithNull.cc @@ -0,0 +1,57 @@ +/* + * 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 "RowConstructorWithNull.h" +#include "velox/expression/VectorFunction.h" + +namespace gluten { +facebook::velox::TypePtr RowConstructorWithNullCallToSpecialForm::resolveType( + const std::vector& argTypes) { + auto numInput = argTypes.size(); + std::vector names(numInput); + std::vector types(numInput); + for (auto i = 0; i < numInput; i++) { + types[i] = argTypes[i]; + names[i] = fmt::format("c{}", i + 1); + } + return facebook::velox::ROW(std::move(names), std::move(types)); +} + +facebook::velox::exec::ExprPtr RowConstructorWithNullCallToSpecialForm::constructSpecialForm( + const std::string& name, + const facebook::velox::TypePtr& type, + std::vector&& compiledChildren, + bool trackCpuUsage, + const facebook::velox::core::QueryConfig& config) { + auto rowConstructorVectorFunction = + facebook::velox::exec::vectorFunctionFactories().withRLock([&config, &name](auto& functionMap) { + auto functionIterator = functionMap.find(name); + return functionIterator->second.factory(name, {}, config); + }); + + return std::make_shared( + type, std::move(compiledChildren), rowConstructorVectorFunction, name, trackCpuUsage); +} + +facebook::velox::exec::ExprPtr RowConstructorWithNullCallToSpecialForm::constructSpecialForm( + const facebook::velox::TypePtr& type, + std::vector&& compiledChildren, + bool trackCpuUsage, + const facebook::velox::core::QueryConfig& config) { + return constructSpecialForm(kRowConstructorWithNull, type, std::move(compiledChildren), trackCpuUsage, config); +} +} // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructorWithNull.h b/cpp/velox/operators/functions/RowConstructorWithNull.h new file mode 100644 index 000000000000..6cfeaee37a6d --- /dev/null +++ b/cpp/velox/operators/functions/RowConstructorWithNull.h @@ -0,0 +1,44 @@ +/* + * 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 "velox/expression/FunctionCallToSpecialForm.h" +#include "velox/expression/SpecialForm.h" + +namespace gluten { +class RowConstructorWithNullCallToSpecialForm : public facebook::velox::exec::FunctionCallToSpecialForm { + public: + facebook::velox::TypePtr resolveType(const std::vector& argTypes) override; + + facebook::velox::exec::ExprPtr constructSpecialForm( + const facebook::velox::TypePtr& type, + std::vector&& compiledChildren, + bool trackCpuUsage, + const facebook::velox::core::QueryConfig& config) override; + + static constexpr const char* kRowConstructorWithNull = "row_constructor_with_null"; + + protected: + facebook::velox::exec::ExprPtr constructSpecialForm( + const std::string& name, + const facebook::velox::TypePtr& type, + std::vector&& compiledChildren, + bool trackCpuUsage, + const facebook::velox::core::QueryConfig& config); +}; +} // namespace gluten diff --git a/cpp/velox/operators/functions/RowConstructor.h b/cpp/velox/operators/functions/RowFunctionWithNull.h similarity index 93% rename from cpp/velox/operators/functions/RowConstructor.h rename to cpp/velox/operators/functions/RowFunctionWithNull.h index a9c83427e099..d66fda99e8ed 100644 --- a/cpp/velox/operators/functions/RowConstructor.h +++ b/cpp/velox/operators/functions/RowFunctionWithNull.h @@ -23,9 +23,9 @@ namespace gluten { /** - * A customized RowConstructor function to set struct as null when one of its argument is null. + * A customized RowFunction to set struct as null when one of its argument is null. */ -class RowConstructor final : public facebook::velox::exec::VectorFunction { +class RowFunctionWithNull final : public facebook::velox::exec::VectorFunction { void apply( const facebook::velox::SelectivityVector& rows, std::vector& args, diff --git a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc index 48f8a8b25143..4306e98b09f1 100644 --- a/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc +++ b/cpp/velox/operators/serializer/VeloxColumnarBatchSerializer.cc @@ -31,10 +31,10 @@ using namespace facebook::velox; namespace gluten { namespace { -std::unique_ptr toByteStream(uint8_t* data, int32_t size) { - auto byteStream = std::make_unique(); - ByteRange byteRange{data, size, 0}; - byteStream->resetInput({byteRange}); +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 diff --git a/cpp/velox/operators/writer/VeloxParquetDatasource.cc b/cpp/velox/operators/writer/VeloxParquetDatasource.cc index 41ed4365edf3..cd004f55065d 100644 --- a/cpp/velox/operators/writer/VeloxParquetDatasource.cc +++ b/cpp/velox/operators/writer/VeloxParquetDatasource.cc @@ -22,41 +22,62 @@ #include #include "arrow/c/bridge.h" -#include "compute/VeloxExecutionCtx.h" +#include "compute/VeloxRuntime.h" #include "config/GlutenConfig.h" #include "utils/VeloxArrowUtils.h" +#include "velox/common/compression/Compression.h" #include "velox/core/QueryConfig.h" #include "velox/core/QueryCtx.h" #include "velox/dwio/common/Options.h" using namespace facebook; using namespace facebook::velox::dwio::common; +using namespace facebook::velox::common; +using namespace facebook::velox::filesystems; namespace gluten { +namespace { +const int32_t kGzipWindowBits4k = 12; +} + void VeloxParquetDatasource::init(const std::unordered_map& sparkConfs) { if (strncmp(filePath_.c_str(), "file:", 5) == 0) { - sink_ = std::make_unique(filePath_.substr(5)); + auto path = filePath_.substr(5); + auto localWriteFile = std::make_unique(path, true, false); + sink_ = std::make_unique(std::move(localWriteFile), path); } else if (strncmp(filePath_.c_str(), "hdfs:", 5) == 0) { #ifdef ENABLE_HDFS - sink_ = std::make_unique(filePath_); + std::string pathSuffix = getHdfsPath(filePath_, HdfsFileSystem::kScheme); + auto fileSystem = getFileSystem(filePath_, nullptr); + auto* hdfsFileSystem = dynamic_cast(fileSystem.get()); + sink_ = std::make_unique(hdfsFileSystem->openFileForWrite(pathSuffix), filePath_); #else throw std::runtime_error( - "The write path is hdfs path but the HDFS haven't been enabled when writing parquet data in velox executionCtx!"); + "The write path is hdfs path but the HDFS haven't been enabled when writing parquet data in velox runtime!"); #endif } else { throw std::runtime_error( - "The file path is not local or hdfs when writing data with parquet format in velox executionCtx!"); + "The file path is not local or hdfs when writing data with parquet format in velox runtime!"); + } + + ArrowSchema cSchema{}; + arrow::Status status = arrow::ExportSchema(*(schema_.get()), &cSchema); + if (!status.ok()) { + throw std::runtime_error("Failed to export arrow cSchema."); } + type_ = velox::importFromArrow(cSchema); + if (sparkConfs.find(kParquetBlockSize) != sparkConfs.end()) { maxRowGroupBytes_ = static_cast(stoi(sparkConfs.find(kParquetBlockSize)->second)); } if (sparkConfs.find(kParquetBlockRows) != sparkConfs.end()) { maxRowGroupRows_ = static_cast(stoi(sparkConfs.find(kParquetBlockRows)->second)); } + velox::parquet::WriterOptions writeOption; auto compressionCodec = CompressionKind::CompressionKind_SNAPPY; if (sparkConfs.find(kParquetCompressionCodec) != sparkConfs.end()) { auto compressionCodecStr = sparkConfs.find(kParquetCompressionCodec)->second; @@ -65,6 +86,14 @@ void VeloxParquetDatasource::init(const std::unordered_mapsecond; + if (parquetGzipWindowSizeStr == kGzipWindowSize4k) { + auto codecOptions = std::make_shared(); + codecOptions->window_bits = kGzipWindowBits4k; + writeOption.codecOptions = std::move(codecOptions); + } + } } else if (boost::iequals(compressionCodecStr, "lzo")) { compressionCodec = CompressionKind::CompressionKind_LZO; } else if (boost::iequals(compressionCodecStr, "brotli")) { @@ -80,16 +109,14 @@ void VeloxParquetDatasource::init(const std::unordered_map configData({{velox::core::QueryConfig::kDataBufferGrowRatio, "2"}}); - auto queryCtx = std::make_shared(nullptr, configData); + writeOption.flushPolicyFactory = [&]() { + return std::make_unique( + maxRowGroupRows_, maxRowGroupBytes_, [&]() { return false; }); + }; + writeOption.schema = gluten::fromArrowSchema(schema_); - parquetWriter_ = std::make_unique(std::move(sink_), writeOption, pool_, schema_); + parquetWriter_ = std::make_unique(std::move(sink_), writeOption, pool_); } void VeloxParquetDatasource::inspectSchema(struct ArrowSchema* out) { @@ -117,7 +144,7 @@ void VeloxParquetDatasource::close() { } void VeloxParquetDatasource::write(const std::shared_ptr& cb) { - auto veloxBatch = VeloxColumnarBatch::from(pool_.get(), cb); + auto veloxBatch = std::dynamic_pointer_cast(cb); VELOX_DCHECK(veloxBatch != nullptr, "Write batch should be VeloxColumnarBatch"); parquetWriter_->write(veloxBatch->getFlattenedRowVector()); } diff --git a/cpp/velox/operators/writer/VeloxParquetDatasource.h b/cpp/velox/operators/writer/VeloxParquetDatasource.h index 809a230f5406..05ab4179a462 100644 --- a/cpp/velox/operators/writer/VeloxParquetDatasource.h +++ b/cpp/velox/operators/writer/VeloxParquetDatasource.h @@ -33,9 +33,10 @@ #include "velox/common/file/FileSystems.h" #ifdef ENABLE_HDFS -#include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSink.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsFileSystem.h" +#include "velox/connectors/hive/storage_adapters/hdfs/HdfsUtil.h" #endif -#include "velox/dwio/common/DataSink.h" +#include "velox/dwio/common/FileSink.h" #include "velox/dwio/common/Options.h" #include "velox/dwio/dwrf/reader/DwrfReader.h" #include "velox/dwio/parquet/writer/Writer.h" @@ -68,7 +69,7 @@ class VeloxParquetDatasource final : public Datasource { std::shared_ptr type_; std::shared_ptr parquetWriter_; std::shared_ptr pool_; - std::unique_ptr sink_; + std::unique_ptr sink_; }; } // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleReader.cc b/cpp/velox/shuffle/VeloxShuffleReader.cc index a9d1406c0442..dfff284ad67c 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.cc +++ b/cpp/velox/shuffle/VeloxShuffleReader.cc @@ -19,10 +19,11 @@ #include +#include "VeloxShuffleUtils.h" #include "memory/VeloxColumnarBatch.h" #include "utils/Common.h" +#include "utils/Compression.h" #include "utils/VeloxArrowUtils.h" -#include "utils/compression.h" #include "utils/macros.h" #include "velox/serializers/PrestoSerializer.h" #include "velox/vector/ComplexVector.h" @@ -120,13 +121,13 @@ VectorPtr readFlatVectorStringView( auto nulls = buffers[bufferIdx++]; auto lengthBuffer = buffers[bufferIdx++]; auto valueBuffer = buffers[bufferIdx++]; - const int32_t* rawLength = lengthBuffer->as(); + const auto* rawLength = lengthBuffer->as(); std::vector stringBuffers; auto values = AlignedBuffer::allocate(sizeof(StringView) * length, pool); auto rawValues = values->asMutable(); auto rawChars = valueBuffer->as(); - auto offset = 0; + uint64_t offset = 0; for (int32_t i = 0; i < length; ++i) { rawValues[i] = StringView(rawChars + offset, rawLength[i]); offset += rawLength[i]; @@ -160,10 +161,10 @@ VectorPtr readFlatVector( return readFlatVectorStringView(buffers, bufferIdx, length, type, pool); } -std::unique_ptr toByteStream(uint8_t* data, int32_t size) { - auto byteStream = std::make_unique(); - ByteRange byteRange{data, size, 0}; - byteStream->resetInput({byteRange}); +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; } @@ -262,6 +263,8 @@ void getUncompressedBuffersOneByOne( codec->Decompress( compressLength, compressBuffer->data(), uncompressLength, uncompressBuffer->mutable_data())); VELOX_DCHECK_EQ(actualDecompressLength, uncompressLength); + // Prevent unused variable warning in optimized build. + ((void)actualDecompressLength); } buffers.emplace_back(convertToVeloxBuffer(uncompressBuffer)); } @@ -314,15 +317,16 @@ void getUncompressedBuffers( const arrow::RecordBatch& batch, arrow::MemoryPool* arrowPool, arrow::util::Codec* codec, - CompressionMode compressionMode, std::vector& buffers) { + // Get compression mode from first byte. auto lengthBuffer = readColumnBuffer(batch, 1); - const int64_t* lengthPtr = reinterpret_cast(lengthBuffer->data()); + auto lengthBufferPtr = reinterpret_cast(lengthBuffer->data()); + auto compressionMode = (CompressionMode)(*lengthBufferPtr++); auto valueBuffer = readColumnBuffer(batch, 2); if (compressionMode == CompressionMode::BUFFER) { - getUncompressedBuffersOneByOne(arrowPool, codec, lengthPtr, valueBuffer, buffers); + getUncompressedBuffersOneByOne(arrowPool, codec, lengthBufferPtr, valueBuffer, buffers); } else { - getUncompressedBuffersStream(arrowPool, codec, lengthPtr, valueBuffer, buffers); + getUncompressedBuffersStream(arrowPool, codec, lengthBufferPtr, valueBuffer, buffers); } } @@ -330,7 +334,6 @@ RowVectorPtr readRowVector( const arrow::RecordBatch& batch, RowTypePtr rowType, CodecBackend codecBackend, - CompressionMode compressionMode, int64_t& decompressTime, int64_t& deserializeTime, arrow::MemoryPool* arrowPool, @@ -352,7 +355,7 @@ RowVectorPtr readRowVector( } else { TIME_NANO_START(decompressTime); auto codec = createArrowIpcCodec(compressType, codecBackend); - getUncompressedBuffers(batch, arrowPool, codec.get(), compressionMode, buffers); + getUncompressedBuffers(batch, arrowPool, codec.get(), buffers); TIME_NANO_END(decompressTime); } @@ -368,7 +371,7 @@ class VeloxShuffleReaderOutStream : public ColumnarBatchIterator { VeloxShuffleReaderOutStream( arrow::MemoryPool* pool, const std::shared_ptr& veloxPool, - const ReaderOptions& options, + const ShuffleReaderOptions& options, const RowTypePtr& rowType, const std::function decompressionTimeAccumulator, const std::function deserializeTimeAccumulator, @@ -391,15 +394,8 @@ class VeloxShuffleReaderOutStream : public ColumnarBatchIterator { int64_t decompressTime = 0LL; int64_t deserializeTime = 0LL; - auto vp = readRowVector( - *rb, - rowType_, - options_.codec_backend, - options_.compression_mode, - decompressTime, - deserializeTime, - pool_, - veloxPool_.get()); + auto vp = + readRowVector(*rb, rowType_, options_.codec_backend, decompressTime, deserializeTime, pool_, veloxPool_.get()); decompressionTimeAccumulator_(decompressTime); deserializeTimeAccumulator_(deserializeTime); @@ -409,7 +405,7 @@ class VeloxShuffleReaderOutStream : public ColumnarBatchIterator { private: arrow::MemoryPool* pool_; std::shared_ptr veloxPool_; - ReaderOptions options_; + ShuffleReaderOptions options_; facebook::velox::RowTypePtr rowType_; std::function decompressionTimeAccumulator_; @@ -418,16 +414,6 @@ class VeloxShuffleReaderOutStream : public ColumnarBatchIterator { ResultIterator in_; }; -std::string getCompressionMode(CompressionMode type) { - if (type == CompressionMode::BUFFER) { - return "BUFFER"; - } else if (type == CompressionMode::ROWVECTOR) { - return "ROWVECTOR"; - } else { - return "UNKNOWN"; - } -} - std::string getCodecBackend(CodecBackend type) { if (type == CodecBackend::QAT) { return "QAT"; @@ -456,7 +442,7 @@ std::string getCompressionType(arrow::Compression::type type) { VeloxShuffleReader::VeloxShuffleReader( std::shared_ptr schema, - ReaderOptions options, + ShuffleReaderOptions options, arrow::MemoryPool* pool, std::shared_ptr veloxPool) : ShuffleReader(schema, options, pool), veloxPool_(std::move(veloxPool)) { @@ -465,7 +451,6 @@ VeloxShuffleReader::VeloxShuffleReader( std::ostringstream oss; oss << "VeloxShuffleReader create, compression_type:" << getCompressionType(options.compression_type); oss << " codec_backend:" << getCodecBackend(options.codec_backend); - oss << " compression_mode:" << getCompressionMode(options.compression_mode); LOG(INFO) << oss.str(); } } diff --git a/cpp/velox/shuffle/VeloxShuffleReader.h b/cpp/velox/shuffle/VeloxShuffleReader.h index 473acf153c23..df1236970db6 100644 --- a/cpp/velox/shuffle/VeloxShuffleReader.h +++ b/cpp/velox/shuffle/VeloxShuffleReader.h @@ -27,7 +27,7 @@ class VeloxShuffleReader final : public ShuffleReader { public: VeloxShuffleReader( std::shared_ptr schema, - ReaderOptions options, + ShuffleReaderOptions options, arrow::MemoryPool* pool, std::shared_ptr veloxPool); diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/utils/DataSourceStrategyUtil.scala b/cpp/velox/shuffle/VeloxShuffleUtils.cc similarity index 53% rename from gluten-core/src/main/scala/org/apache/spark/sql/utils/DataSourceStrategyUtil.scala rename to cpp/velox/shuffle/VeloxShuffleUtils.cc index e361a4a1cb58..d4a7fd317c38 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/utils/DataSourceStrategyUtil.scala +++ b/cpp/velox/shuffle/VeloxShuffleUtils.cc @@ -14,20 +14,29 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql.utils -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.execution.datasources.DataSourceStrategy -import org.apache.spark.sql.sources.Filter +#include "shuffle/VeloxShuffleUtils.h" +#include +#include -object DataSourceStrategyUtil { +int64_t gluten::getMaxCompressedBufferSize( + const std::vector>& buffers, + arrow::util::Codec* codec) { + int64_t totalSize = 0; + for (auto& buffer : buffers) { + if (buffer != nullptr && buffer->size() != 0) { + totalSize += codec->MaxCompressedLen(buffer->size(), nullptr); + } + } + return totalSize; +} - /** - * Translates a runtime filter into a data source filter. - * - * Runtime filters usually contain a subquery that must be evaluated before the translation. If - * the underlying subquery hasn't completed yet, this method will throw an exception. - */ - def translateRuntimeFilter(expr: Expression): Option[Filter] = - DataSourceStrategy.translateRuntimeFilter(expr) +int64_t gluten::getBuffersSize(const std::vector>& buffers) { + int64_t totalSize = 0; + for (auto& buffer : buffers) { + if (buffer != nullptr) { + totalSize += buffer->size(); + } + } + return totalSize; } diff --git a/cpp/velox/shuffle/VeloxShuffleUtils.h b/cpp/velox/shuffle/VeloxShuffleUtils.h new file mode 100644 index 000000000000..d679933ec34b --- /dev/null +++ b/cpp/velox/shuffle/VeloxShuffleUtils.h @@ -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. + */ + +#pragma once + +#include + +namespace gluten { + +using BinaryArrayLengthBufferType = uint32_t; +using IpcOffsetBufferType = arrow::LargeStringType::offset_type; + +static const size_t kSizeOfBinaryArrayLengthBuffer = sizeof(BinaryArrayLengthBufferType); +static const size_t kSizeOfIpcOffsetBuffer = sizeof(IpcOffsetBufferType); + +int64_t getBuffersSize(const std::vector>& buffers); + +int64_t getMaxCompressedBufferSize( + const std::vector>& buffers, + arrow::util::Codec* codec); + +} // namespace gluten diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.cc b/cpp/velox/shuffle/VeloxShuffleWriter.cc index 414e03212ea5..2329352a94b5 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.cc +++ b/cpp/velox/shuffle/VeloxShuffleWriter.cc @@ -16,17 +16,22 @@ */ #include "VeloxShuffleWriter.h" +#include "VeloxShuffleUtils.h" #include "memory/ArrowMemory.h" #include "memory/VeloxColumnarBatch.h" #include "memory/VeloxMemoryManager.h" -#include "utils/VeloxArrowUtils.h" -#include "velox/vector/arrow/Bridge.h" - +#include "shuffle/Partitioner.h" #include "utils/Common.h" -#include "utils/compression.h" +#include "utils/Compression.h" +#include "utils/Timer.h" +#include "utils/VeloxArrowUtils.h" #include "utils/macros.h" - -#include "arrow/c/bridge.h" +#include "velox/buffer/Buffer.h" +#include "velox/type/HugeInt.h" +#include "velox/type/Timestamp.h" +#include "velox/type/Type.h" +#include "velox/vector/BaseVector.h" +#include "velox/vector/ComplexVector.h" #if defined(__x86_64__) #include @@ -35,11 +40,6 @@ #include #endif -#include - -using namespace facebook; -using namespace facebook::velox; - namespace gluten { #define VELOX_SHUFFLE_WRITER_LOG_FLAG 0 @@ -61,44 +61,22 @@ namespace gluten { #define PREFETCHT1(ptr) __builtin_prefetch(ptr, 0, 2) #define PREFETCHT2(ptr) __builtin_prefetch(ptr, 0, 1) #endif -// #define SKIPWRITE - -#if defined(__x86_64__) -template -std::string m128iToString(const __m128i var) { - std::stringstream sstr; - T values[16 / sizeof(T)]; - std::memcpy(values, &var, sizeof(values)); // See discussion below - if (sizeof(T) == 1) { - for (unsigned int i = 0; i < sizeof(__m128i); i++) { // C++11: Range for also possible - sstr << std::hex << (int)values[i] << " " << std::dec; - } - } else { - for (unsigned int i = 0; i < sizeof(__m128i) / sizeof(T); i++) { - sstr << std::hex << values[i] << " " << std::dec; - } - } - return sstr.str(); -} -#endif namespace { -using BinaryArrayOffsetType = arrow::BinaryType::offset_type; - -bool vectorHasNull(const velox::VectorPtr& vp) { +bool vectorHasNull(const facebook::velox::VectorPtr& vp) { if (!vp->mayHaveNulls()) { return false; } return vp->countNulls(vp->nulls(), vp->size()) != 0; } -velox::RowVectorPtr getStrippedRowVector(const velox::RowVector& rv) { +facebook::velox::RowVectorPtr getStrippedRowVector(const facebook::velox::RowVector& rv) { // get new row type auto rowType = rv.type()->asRow(); auto typeChildren = rowType.children(); typeChildren.erase(typeChildren.begin()); - auto newRowType = velox::ROW(std::move(typeChildren)); + auto newRowType = facebook::velox::ROW(std::move(typeChildren)); // get length auto length = rv.size(); @@ -107,34 +85,36 @@ velox::RowVectorPtr getStrippedRowVector(const velox::RowVector& rv) { auto children = rv.children(); children.erase(children.begin()); - return std::make_shared(rv.pool(), newRowType, BufferPtr(nullptr), length, std::move(children)); + return std::make_shared( + rv.pool(), newRowType, facebook::velox::BufferPtr(nullptr), length, std::move(children)); } -const int32_t* getFirstColumn(const velox::RowVector& rv) { - VELOX_DCHECK(rv.childrenSize() > 0, "RowVector missing partition id column."); +const int32_t* getFirstColumn(const facebook::velox::RowVector& rv) { + VELOX_CHECK(rv.childrenSize() > 0, "RowVector missing partition id column."); auto& firstChild = rv.childAt(0); - VELOX_DCHECK(firstChild->type()->isInteger(), "RecordBatch field 0 should be integer"); + VELOX_CHECK(firstChild->type()->isInteger(), "RecordBatch field 0 should be integer"); // first column is partition key hash value or pid return firstChild->asFlatVector()->rawValues(); } -std::shared_ptr makeNullBinaryArray(std::shared_ptr type, arrow::MemoryPool* pool) { - size_t sizeofBinaryOffset = sizeof(arrow::LargeStringType::offset_type); - GLUTEN_ASSIGN_OR_THROW(auto offsetBuffer, arrow::AllocateResizableBuffer(sizeofBinaryOffset << 1, pool)); +arrow::Result> makeNullBinaryArray( + std::shared_ptr type, + arrow::MemoryPool* pool) { + ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); // set the first offset to 0, and set the value offset uint8_t* offsetaddr = offsetBuffer->mutable_data(); - memset(offsetaddr, 0, sizeofBinaryOffset); + memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); // second value offset 0 - memset(offsetaddr + sizeofBinaryOffset, 0, sizeofBinaryOffset); + memset(offsetaddr + kSizeOfIpcOffsetBuffer, 0, kSizeOfIpcOffsetBuffer); // If it is not compressed array, null valueBuffer // worked, but if compress, will core dump at buffer::size(), so replace by kNullBuffer static std::shared_ptr kNullBuffer = std::make_shared(nullptr, 0); return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), kNullBuffer})); } -std::shared_ptr makeBinaryArray( +arrow::Result> makeBinaryArray( std::shared_ptr type, std::shared_ptr valueBuffer, arrow::MemoryPool* pool) { @@ -142,184 +122,143 @@ std::shared_ptr makeBinaryArray( return makeNullBinaryArray(type, pool); } - size_t sizeofBinaryOffset = sizeof(arrow::LargeStringType::offset_type); - GLUTEN_ASSIGN_OR_THROW(auto offsetBuffer, arrow::AllocateResizableBuffer(sizeofBinaryOffset << 1, pool)); + ARROW_ASSIGN_OR_RAISE(auto offsetBuffer, arrow::AllocateResizableBuffer(kSizeOfIpcOffsetBuffer << 1, pool)); // set the first offset to 0, and set the value offset uint8_t* offsetaddr = offsetBuffer->mutable_data(); - memset(offsetaddr, 0, sizeofBinaryOffset); + memset(offsetaddr, 0, kSizeOfIpcOffsetBuffer); int64_t length = valueBuffer->size(); - memcpy(offsetaddr + sizeofBinaryOffset, reinterpret_cast(&length), sizeofBinaryOffset); + memcpy(offsetaddr + kSizeOfIpcOffsetBuffer, reinterpret_cast(&length), kSizeOfIpcOffsetBuffer); return arrow::MakeArray(arrow::ArrayData::Make(type, 1, {nullptr, std::move(offsetBuffer), valueBuffer})); } -inline void writeInt64(std::shared_ptr buffer, int64_t& offset, int64_t value) { - memcpy(buffer->mutable_data() + offset, &value, sizeof(int64_t)); - offset += sizeof(int64_t); -} - -int64_t getMaxCompressedBufferSize( - const std::vector>& buffers, - arrow::util::Codec* codec) { - int64_t totalSize = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr && buffer->size() != 0) { - totalSize += codec->MaxCompressedLen(buffer->size(), nullptr); - } - } - return totalSize; -} - -// Length buffer layout |buffers.size()|buffer1 unCompressedLength|buffer1 compressedLength| buffer2... -void getLengthBufferAndValueBufferOneByOne( +// Length buffer layout |compressionMode|buffers.size()|buffer1 unCompressedLength|buffer1 compressedLength| buffer2... +arrow::Status getLengthBufferAndValueBufferOneByOne( const std::vector>& buffers, arrow::MemoryPool* pool, arrow::util::Codec* codec, - int32_t bufferCompressThreshold, std::shared_ptr& lengthBuffer, std::shared_ptr& valueBuffer) { - GLUTEN_ASSIGN_OR_THROW( - lengthBuffer, arrow::AllocateResizableBuffer((buffers.size() * 2 + 1) * sizeof(int64_t), pool)); - int64_t offset = 0; - writeInt64(lengthBuffer, offset, buffers.size()); + ARROW_ASSIGN_OR_RAISE( + lengthBuffer, arrow::AllocateResizableBuffer((1 + 1 + buffers.size() * 2) * sizeof(int64_t), pool)); + auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); + // Write compression mode. + *lengthBufferPtr++ = CompressionMode::BUFFER; + // Write number of buffers. + *lengthBufferPtr++ = buffers.size(); int64_t compressedBufferMaxSize = getMaxCompressedBufferSize(buffers, codec); - GLUTEN_ASSIGN_OR_THROW(valueBuffer, arrow::AllocateResizableBuffer(compressedBufferMaxSize, pool)); + ARROW_ASSIGN_OR_RAISE(valueBuffer, arrow::AllocateResizableBuffer(compressedBufferMaxSize, pool)); int64_t compressValueOffset = 0; for (auto& buffer : buffers) { if (buffer != nullptr && buffer->size() != 0) { int64_t actualLength; - if (buffer->size() >= bufferCompressThreshold) { - writeInt64(lengthBuffer, offset, buffer->size()); - int64_t maxLength = codec->MaxCompressedLen(buffer->size(), nullptr); - GLUTEN_ASSIGN_OR_THROW( - actualLength, - codec->Compress( - buffer->size(), buffer->data(), maxLength, valueBuffer->mutable_data() + compressValueOffset)); - } else { - // Will not compress small buffer, mark uncompressed length as -1 to indicate it is original buffer - writeInt64(lengthBuffer, offset, -1); - gluten::fastCopy(valueBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); - actualLength = buffer->size(); - } + int64_t maxLength = codec->MaxCompressedLen(buffer->size(), nullptr); + ARROW_ASSIGN_OR_RAISE( + actualLength, + codec->Compress( + buffer->size(), buffer->data(), maxLength, valueBuffer->mutable_data() + compressValueOffset)); compressValueOffset += actualLength; - writeInt64(lengthBuffer, offset, actualLength); + *lengthBufferPtr++ = buffer->size(); + *lengthBufferPtr++ = actualLength; } else { - writeInt64(lengthBuffer, offset, 0); - writeInt64(lengthBuffer, offset, 0); - } - } - GLUTEN_THROW_NOT_OK(valueBuffer->Resize(compressValueOffset, /*shrink*/ true)); -} - -int64_t getBuffersSize(const std::vector>& buffers) { - int64_t totalSize = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr) { - totalSize += buffer->size(); + *lengthBufferPtr++ = 0; + *lengthBufferPtr++ = 0; } } - return totalSize; + RETURN_NOT_OK(valueBuffer->Resize(compressValueOffset, /*shrink*/ true)); + return arrow::Status::OK(); } -// Length buffer layout |buffer unCompressedLength|buffer compressedLength|buffers.size()| buffer1 size | buffer2 size -void getLengthBufferAndValueBufferStream( +// Length buffer layout |compressionMode|buffer unCompressedLength|buffer compressedLength|buffers.size()| buffer1 size +// | buffer2 size +arrow::Status getLengthBufferAndValueBufferStream( const std::vector>& buffers, arrow::MemoryPool* pool, arrow::util::Codec* codec, - int32_t bufferCompressThreshold, std::shared_ptr& lengthBuffer, std::shared_ptr& compressedBuffer) { - GLUTEN_ASSIGN_OR_THROW(lengthBuffer, arrow::AllocateResizableBuffer((buffers.size() + 3) * sizeof(int64_t), pool)); - + ARROW_ASSIGN_OR_RAISE(lengthBuffer, arrow::AllocateResizableBuffer((1 + 3 + buffers.size()) * sizeof(int64_t), pool)); auto originalBufferSize = getBuffersSize(buffers); - if (originalBufferSize >= bufferCompressThreshold) { - // because 64B align, uncompressedBuffer size maybe bigger than unCompressedBufferSize which is - // getBuffersSize(buffers), then cannot use this size - GLUTEN_ASSIGN_OR_THROW(auto uncompressedBuffer, arrow::AllocateResizableBuffer(originalBufferSize, pool)); - int64_t uncompressedSize = uncompressedBuffer->size(); - - // Write metadata. - auto lengthBufferPtr = (int64_t*)lengthBuffer->mutable_data(); - int64_t pos = 0; - lengthBufferPtr[pos++] = uncompressedSize; // uncompressedLength - lengthBufferPtr[pos++] = 0; // 0 for compressedLength - lengthBufferPtr[pos++] = buffers.size(); - - int64_t compressValueOffset = 0; - for (auto& buffer : buffers) { - // Copy all buffers into one big buffer. - if (buffer != nullptr && buffer->size() != 0) { - lengthBufferPtr[pos++] = buffer->size(); - gluten::fastCopy(uncompressedBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); - compressValueOffset += buffer->size(); - } else { - lengthBufferPtr[pos++] = 0; - } - } - - // Compress the big buffer. - int64_t maxLength = codec->MaxCompressedLen(uncompressedSize, nullptr); - GLUTEN_ASSIGN_OR_THROW(compressedBuffer, arrow::AllocateResizableBuffer(maxLength, pool)); - GLUTEN_ASSIGN_OR_THROW( - int64_t actualLength, - codec->Compress(uncompressedSize, uncompressedBuffer->data(), maxLength, compressedBuffer->mutable_data())); - GLUTEN_THROW_NOT_OK(compressedBuffer->Resize(actualLength, /*shrink*/ true)); + // because 64B align, uncompressedBuffer size maybe bigger than unCompressedBufferSize which is + // getBuffersSize(buffers), then cannot use this size + ARROW_ASSIGN_OR_RAISE(auto uncompressedBuffer, arrow::AllocateResizableBuffer(originalBufferSize, pool)); + int64_t uncompressedSize = uncompressedBuffer->size(); + + auto lengthBufferPtr = (int64_t*)(lengthBuffer->mutable_data()); + // First write metadata. + // Write compression mode. + *lengthBufferPtr++ = CompressionMode::ROWVECTOR; + // Store uncompressed size. + *lengthBufferPtr++ = uncompressedSize; // uncompressedLength + // Skip compressed size and update later. + auto compressedLengthPtr = lengthBufferPtr++; + // Store number of buffers. + *lengthBufferPtr++ = buffers.size(); - // Update compressedLength. - lengthBufferPtr[1] = actualLength; - } else { - int64_t offset = 0; - // mark uncompress size as -1 to mark it is uncompressed buffer - writeInt64(lengthBuffer, offset, -1); // unCompressedBufferSize - GLUTEN_ASSIGN_OR_THROW(compressedBuffer, arrow::AllocateResizableBuffer(originalBufferSize, pool)); - writeInt64(lengthBuffer, offset, compressedBuffer->size()); // 0 for compressLength - writeInt64(lengthBuffer, offset, buffers.size()); - int64_t compressValueOffset = 0; - for (auto& buffer : buffers) { - if (buffer != nullptr && buffer->size() != 0) { - writeInt64(lengthBuffer, offset, buffer->size()); - gluten::fastCopy(compressedBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); - compressValueOffset += buffer->size(); - } else { - writeInt64(lengthBuffer, offset, 0); - } + int64_t compressValueOffset = 0; + for (auto& buffer : buffers) { + // Copy all buffers into one big buffer. + if (buffer != nullptr && buffer->size() != 0) { + *lengthBufferPtr++ = buffer->size(); + gluten::fastCopy(uncompressedBuffer->mutable_data() + compressValueOffset, buffer->data(), buffer->size()); + compressValueOffset += buffer->size(); + } else { + *lengthBufferPtr++ = 0; } } + + // Compress the big buffer. + int64_t maxLength = codec->MaxCompressedLen(uncompressedSize, nullptr); + ARROW_ASSIGN_OR_RAISE(compressedBuffer, arrow::AllocateResizableBuffer(maxLength, pool)); + ARROW_ASSIGN_OR_RAISE( + int64_t actualLength, + codec->Compress(uncompressedSize, uncompressedBuffer->data(), maxLength, compressedBuffer->mutable_data())); + RETURN_NOT_OK(compressedBuffer->Resize(actualLength, /*shrink*/ true)); + + // Update compressed size. + *compressedLengthPtr = actualLength; + return arrow::Status::OK(); } -std::shared_ptr makeCompressedRecordBatch( +arrow::Result> makeCompressedRecordBatch( uint32_t numRows, const std::vector>& buffers, const std::shared_ptr compressWriteSchema, arrow::MemoryPool* pool, arrow::util::Codec* codec, int32_t bufferCompressThreshold, - CompressionMode compressionMode) { + CompressionMode compressionMode, + int64_t& compressionTime) { + ScopedTimer{compressionTime}; std::vector> arrays; // header col, numRows, compressionType { - GLUTEN_ASSIGN_OR_THROW(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); + ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); int32_t compressType = static_cast(codec->compression_type()); memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); - arrays.emplace_back(makeBinaryArray(compressWriteSchema->field(0)->type(), std::move(headerBuffer), pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE( + arrays.back(), makeBinaryArray(compressWriteSchema->field(0)->type(), std::move(headerBuffer), pool)); } std::shared_ptr lengthBuffer; std::shared_ptr valueBuffer; - if (compressionMode == CompressionMode::BUFFER) { - getLengthBufferAndValueBufferOneByOne(buffers, pool, codec, bufferCompressThreshold, lengthBuffer, valueBuffer); + if (compressionMode == CompressionMode::BUFFER && numRows > bufferCompressThreshold) { + RETURN_NOT_OK(getLengthBufferAndValueBufferOneByOne(buffers, pool, codec, lengthBuffer, valueBuffer)); } else { - getLengthBufferAndValueBufferStream(buffers, pool, codec, bufferCompressThreshold, lengthBuffer, valueBuffer); + RETURN_NOT_OK(getLengthBufferAndValueBufferStream(buffers, pool, codec, lengthBuffer, valueBuffer)); } - arrays.emplace_back(makeBinaryArray(compressWriteSchema->field(1)->type(), lengthBuffer, pool)); - arrays.emplace_back(makeBinaryArray(compressWriteSchema->field(2)->type(), valueBuffer, pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(1)->type(), lengthBuffer, pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(compressWriteSchema->field(2)->type(), valueBuffer, pool)); return arrow::RecordBatch::Make(compressWriteSchema, 1, {arrays}); } // generate the new big one row several columns binary recordbatch -std::shared_ptr makeUncompressedRecordBatch( +arrow::Result> makeUncompressedRecordBatch( uint32_t numRows, const std::vector>& buffers, const std::shared_ptr writeSchema, @@ -327,22 +266,105 @@ std::shared_ptr makeUncompressedRecordBatch( std::vector> arrays; // header col, numRows, compressionType { - GLUTEN_ASSIGN_OR_THROW(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); + ARROW_ASSIGN_OR_RAISE(auto headerBuffer, arrow::AllocateResizableBuffer(sizeof(uint32_t) + sizeof(int32_t), pool)); memcpy(headerBuffer->mutable_data(), &numRows, sizeof(uint32_t)); int32_t compressType = static_cast(arrow::Compression::type::UNCOMPRESSED); memcpy(headerBuffer->mutable_data() + sizeof(uint32_t), &compressType, sizeof(int32_t)); - arrays.emplace_back(makeBinaryArray(writeSchema->field(0)->type(), std::move(headerBuffer), pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(0)->type(), std::move(headerBuffer), pool)); } int32_t bufferNum = writeSchema->num_fields() - 1; for (int32_t i = 0; i < bufferNum; i++) { - arrays.emplace_back(makeBinaryArray(writeSchema->field(i + 1)->type(), buffers[i], pool)); + arrays.emplace_back(); + ARROW_ASSIGN_OR_RAISE(arrays.back(), makeBinaryArray(writeSchema->field(i + 1)->type(), buffers[i], pool)); } return arrow::RecordBatch::Make(writeSchema, 1, {arrays}); } + +class EvictGuard { + public: + explicit EvictGuard(EvictState& evictState) : evictState_(evictState) { + evictState_ = EvictState::kUnevictable; + } + + ~EvictGuard() { + evictState_ = EvictState::kEvictable; + } + + // For safety and clarity. + EvictGuard(const EvictGuard&) = delete; + EvictGuard& operator=(const EvictGuard&) = delete; + EvictGuard(EvictGuard&&) = delete; + EvictGuard& operator=(EvictGuard&&) = delete; + + private: + EvictState& evictState_; +}; + +template +arrow::Status collectFlatVectorBuffer( + facebook::velox::BaseVector* vector, + std::vector>& buffers, + arrow::MemoryPool* pool) { + using T = typename facebook::velox::TypeTraits::NativeType; + auto flatVector = dynamic_cast*>(vector); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), toArrowBuffer(flatVector->nulls(), pool)); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), toArrowBuffer(flatVector->values(), pool)); + return arrow::Status::OK(); +} + +arrow::Status collectFlatVectorBufferStringView( + facebook::velox::BaseVector* vector, + std::vector>& buffers, + arrow::MemoryPool* pool) { + auto flatVector = dynamic_cast*>(vector); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), toArrowBuffer(flatVector->nulls(), pool)); + + auto rawValues = flatVector->rawValues(); + // last offset is the totalStringSize + auto lengthBufferSize = sizeof(gluten::BinaryArrayLengthBufferType) * flatVector->size(); + ARROW_ASSIGN_OR_RAISE(auto lengthBuffer, arrow::AllocateResizableBuffer(lengthBufferSize, pool)); + auto* rawLength = reinterpret_cast(lengthBuffer->mutable_data()); + uint64_t offset = 0; + for (int32_t i = 0; i < flatVector->size(); i++) { + auto length = rawValues[i].size(); + *rawLength++ = length; + offset += length; + } + buffers.push_back(std::move(lengthBuffer)); + + ARROW_ASSIGN_OR_RAISE(auto valueBuffer, arrow::AllocateResizableBuffer(offset, pool)); + auto raw = reinterpret_cast(valueBuffer->mutable_data()); + for (int32_t i = 0; i < flatVector->size(); i++) { + gluten::fastCopy(raw, rawValues[i].data(), rawValues[i].size()); + raw += rawValues[i].size(); + } + buffers.push_back(std::move(valueBuffer)); + return arrow::Status::OK(); +} + +template <> +arrow::Status collectFlatVectorBuffer( + facebook::velox::BaseVector* vector, + std::vector>& buffers, + arrow::MemoryPool* pool) { + return collectFlatVectorBufferStringView(vector, buffers, pool); +} + +template <> +arrow::Status collectFlatVectorBuffer( + facebook::velox::BaseVector* vector, + std::vector>& buffers, + arrow::MemoryPool* pool) { + return collectFlatVectorBufferStringView(vector, buffers, pool); +} + } // namespace -// VeloxShuffleWriter arrow::Result> VeloxShuffleWriter::create( uint32_t numPartitions, std::shared_ptr partitionWriterCreator, @@ -353,7 +375,7 @@ arrow::Result> VeloxShuffleWriter::create( oss << "Velox shuffle writer created,"; oss << " partitionNum:" << numPartitions; oss << " partitionWriterCreator:" << typeid(*partitionWriterCreator.get()).name(); - oss << " partitioning_name:" << options.partitioning_name; + oss << " partitioning:" << options.partitioning; oss << " buffer_size:" << options.buffer_size; oss << " compression_type:" << (int)options.compression_type; oss << " codec_backend:" << (int)options.codec_backend; @@ -371,6 +393,8 @@ arrow::Result> VeloxShuffleWriter::create( } arrow::Status VeloxShuffleWriter::init() { + RETURN_NOT_OK(initIpcWriteOptions()); + #if defined(__x86_64__) supportAvx512_ = __builtin_cpu_supports("avx512bw"); #else @@ -383,12 +407,15 @@ arrow::Status VeloxShuffleWriter::init() { // split record batch size should be less than 32k VELOX_CHECK_LE(options_.buffer_size, 32 * 1024); + // memory_pool should be assigned. + VELOX_CHECK_NOT_NULL(options_.memory_pool); + ARROW_ASSIGN_OR_RAISE(partitionWriter_, partitionWriterCreator_->make(this)); - ARROW_ASSIGN_OR_RAISE(partitioner_, Partitioner::make(options_.partitioning_name, numPartitions_)); + ARROW_ASSIGN_OR_RAISE(partitioner_, Partitioner::make(options_.partitioning, numPartitions_)); // pre-allocated buffer size for each partition, unit is row count // when partitioner is SinglePart, partial variables don`t need init - if (options_.partitioning_name != "single") { + if (options_.partitioning != Partitioning::kSingle) { partition2RowCount_.resize(numPartitions_); partition2BufferSize_.resize(numPartitions_); partition2RowOffset_.resize(numPartitions_ + 1); @@ -399,16 +426,12 @@ arrow::Status VeloxShuffleWriter::init() { partitionLengths_.resize(numPartitions_); rawPartitionLengths_.resize(numPartitions_); - RETURN_NOT_OK(initIpcWriteOptions()); - return arrow::Status::OK(); } arrow::Status VeloxShuffleWriter::initIpcWriteOptions() { - auto& ipcWriteOptions = options_.ipc_write_options; - ipcWriteOptions.memory_pool = payloadPool_.get(); - ipcWriteOptions.use_threads = false; - + options_.ipc_write_options.memory_pool = payloadPool_.get(); + options_.ipc_write_options.use_threads = false; return arrow::Status::OK(); } @@ -437,129 +460,66 @@ arrow::Status VeloxShuffleWriter::initPartitions() { return arrow::Status::OK(); } -namespace { - -std::shared_ptr convertToArrowBuffer(velox::BufferPtr buffer, arrow::MemoryPool* pool) { - if (buffer == nullptr) { - return nullptr; - } - - GLUTEN_ASSIGN_OR_THROW(auto arrowBuffer, arrow::AllocateResizableBuffer(buffer->size(), pool)); - gluten::fastCopy(arrowBuffer->mutable_data(), buffer->asMutable(), buffer->size()); - return arrowBuffer; -} - -template -void collectFlatVectorBuffer( - BaseVector* vector, - std::vector>& buffers, - arrow::MemoryPool* pool) { - using T = typename velox::TypeTraits::NativeType; - auto flatVector = dynamic_cast*>(vector); - buffers.emplace_back(convertToArrowBuffer(flatVector->nulls(), pool)); - buffers.emplace_back(convertToArrowBuffer(flatVector->values(), pool)); -} - -void collectFlatVectorBufferStringView( - BaseVector* vector, - std::vector>& buffers, - arrow::MemoryPool* pool) { - auto flatVector = dynamic_cast*>(vector); - buffers.emplace_back(convertToArrowBuffer(flatVector->nulls(), pool)); - - auto rawValues = flatVector->rawValues(); - // last offset is the totalStringSize - auto lengthBufferSize = sizeof(int32_t) * flatVector->size(); - GLUTEN_ASSIGN_OR_THROW(auto lengthBuffer, arrow::AllocateResizableBuffer(lengthBufferSize, pool)); - int32_t* rawLength = reinterpret_cast(lengthBuffer->mutable_data()); - int32_t offset = 0; - for (int32_t i = 0; i < flatVector->size(); i++) { - auto length = rawValues[i].size(); - *rawLength++ = length; - offset += length; - } - buffers.push_back(std::move(lengthBuffer)); - - GLUTEN_ASSIGN_OR_THROW(auto valueBuffer, arrow::AllocateResizableBuffer(offset, pool)); - auto raw = reinterpret_cast(valueBuffer->mutable_data()); - for (int32_t i = 0; i < flatVector->size(); i++) { - gluten::fastCopy(raw, rawValues[i].data(), rawValues[i].size()); - raw += rawValues[i].size(); - } - buffers.push_back(std::move(valueBuffer)); -} - -template <> -void collectFlatVectorBuffer( - BaseVector* vector, - std::vector>& buffers, - arrow::MemoryPool* pool) { - collectFlatVectorBufferStringView(vector, buffers, pool); -} - -template <> -void collectFlatVectorBuffer( - BaseVector* vector, - std::vector>& buffers, - arrow::MemoryPool* pool) { - collectFlatVectorBufferStringView(vector, buffers, pool); -} - -} // namespace - -std::shared_ptr VeloxShuffleWriter::generateComplexTypeBuffers(velox::RowVectorPtr vector) { - auto arena = std::make_unique(veloxPool_.get()); +arrow::Result> VeloxShuffleWriter::generateComplexTypeBuffers( + facebook::velox::RowVectorPtr vector) { + auto arena = std::make_unique(veloxPool_.get()); auto serializer = serde_.createSerializer(asRowType(vector->type()), vector->size(), arena.get(), /* serdeOptions */ nullptr); - const IndexRange allRows{0, vector->size()}; + const facebook::velox::IndexRange allRows{0, vector->size()}; serializer->append(vector, folly::Range(&allRows, 1)); auto serializedSize = serializer->maxSerializedSize(); auto flushBuffer = complexTypeFlushBuffer_[0]; if (flushBuffer == nullptr) { - GLUTEN_ASSIGN_OR_THROW(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); + ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); } else if (serializedSize > flushBuffer->capacity()) { - GLUTEN_THROW_NOT_OK(flushBuffer->Reserve(serializedSize)); + RETURN_NOT_OK(flushBuffer->Reserve(serializedSize)); } auto valueBuffer = arrow::SliceMutableBuffer(flushBuffer, 0, serializedSize); auto output = std::make_shared(valueBuffer); - serializer::presto::PrestoOutputStreamListener listener; + facebook::velox::serializer::presto::PrestoOutputStreamListener listener; ArrowFixedSizeBufferOutputStream out(output, &listener); serializer->flush(&out); return valueBuffer; } arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64_t memLimit) { - if (options_.partitioning_name == "single") { + if (options_.partitioning == Partitioning::kSingle) { auto veloxColumnBatch = VeloxColumnarBatch::from(veloxPool_.get(), cb); - VELOX_DCHECK_NOT_NULL(veloxColumnBatch); + VELOX_CHECK_NOT_NULL(veloxColumnBatch); auto& rv = *veloxColumnBatch->getFlattenedRowVector(); RETURN_NOT_OK(initFromRowVector(rv)); std::vector> buffers; - std::vector complexChildren; + std::vector complexChildren; for (auto& child : rv.children()) { - if (child->encoding() == VectorEncoding::Simple::FLAT) { - VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( + if (child->encoding() == facebook::velox::VectorEncoding::Simple::FLAT) { + auto status = VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH_ALL( collectFlatVectorBuffer, child->typeKind(), child.get(), buffers, payloadPool_.get()); + RETURN_NOT_OK(status); } else { complexChildren.emplace_back(child); } } if (complexChildren.size() > 0) { - auto rowVector = std::make_shared( - veloxPool_.get(), complexWriteType_, BufferPtr(nullptr), rv.size(), std::move(complexChildren)); - buffers.emplace_back(generateComplexTypeBuffers(rowVector)); + auto rowVector = std::make_shared( + veloxPool_.get(), + complexWriteType_, + facebook::velox::BufferPtr(nullptr), + rv.size(), + std::move(complexChildren)); + buffers.emplace_back(); + ARROW_ASSIGN_OR_RAISE(buffers.back(), generateComplexTypeBuffers(rowVector)); } rawPartitionLengths_[0] += getBuffersSize(buffers); - auto rb = makeRecordBatch(rv.size(), buffers); - ARROW_ASSIGN_OR_RAISE(auto payload, createArrowIpcPayload(*rb, false)); - RETURN_NOT_OK(partitionWriter_->processPayload(0, std::move(payload))); - } else if (options_.partitioning_name == "range") { + ARROW_ASSIGN_OR_RAISE(auto rb, makeRecordBatch(rv.size(), buffers)); + ARROW_ASSIGN_OR_RAISE(auto payload, createPayload(*rb, false)); + RETURN_NOT_OK(evictPayload(0, std::move(payload))); + } else if (options_.partitioning == Partitioning::kRange) { auto compositeBatch = std::dynamic_pointer_cast(cb); - VELOX_DCHECK_NOT_NULL(compositeBatch); + VELOX_CHECK_NOT_NULL(compositeBatch); auto batches = compositeBatch->getBatches(); - VELOX_DCHECK_EQ(batches.size(), 2); + VELOX_CHECK_EQ(batches.size(), 2); auto pidBatch = VeloxColumnarBatch::from(veloxPool_.get(), batches[0]); auto pidArr = getFirstColumn(*(pidBatch->getRowVector())); START_TIMING(cpuWallTimingList_[CpuWallTimingCompute]); @@ -571,8 +531,8 @@ arrow::Status VeloxShuffleWriter::split(std::shared_ptr cb, int64 RETURN_NOT_OK(doSplit(rv, memLimit)); } else { auto veloxColumnBatch = VeloxColumnarBatch::from(veloxPool_.get(), cb); - VELOX_DCHECK_NOT_NULL(veloxColumnBatch); - velox::RowVectorPtr rv; + VELOX_CHECK_NOT_NULL(veloxColumnBatch); + facebook::velox::RowVectorPtr rv; START_TIMING(cpuWallTimingList_[CpuWallTimingFlattenRV]); rv = veloxColumnBatch->getFlattenedRowVector(); END_TIMING(); @@ -599,10 +559,8 @@ arrow::Status VeloxShuffleWriter::stop() { { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingStop]); setSplitState(SplitState::kStop); - EVAL_START("write", options_.thread_id) RETURN_NOT_OK(partitionWriter_->stop()); partitionBuffers_.clear(); - EVAL_END("write", options_.thread_id, options_.task_attempt_id) } stat(); @@ -643,7 +601,7 @@ arrow::Status VeloxShuffleWriter::buildPartition2Row(uint32_t rowNum) { return arrow::Status::OK(); } -arrow::Status VeloxShuffleWriter::updateInputHasNull(const velox::RowVector& rv) { +arrow::Status VeloxShuffleWriter::updateInputHasNull(const facebook::velox::RowVector& rv) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingHasNull]); for (size_t col = 0; col < simpleColumnIndices_.size(); ++col) { @@ -662,12 +620,17 @@ arrow::Status VeloxShuffleWriter::updateInputHasNull(const velox::RowVector& rv) return arrow::Status::OK(); } -arrow::Status VeloxShuffleWriter::doSplit(const velox::RowVector& rv, int64_t memLimit) { +void VeloxShuffleWriter::setSplitState(SplitState state) { + splitState_ = state; +} + +arrow::Status VeloxShuffleWriter::doSplit(const facebook::velox::RowVector& rv, int64_t memLimit) { auto rowNum = rv.size(); RETURN_NOT_OK(buildPartition2Row(rowNum)); RETURN_NOT_OK(updateInputHasNull(rv)); START_TIMING(cpuWallTimingList_[CpuWallTimingIteratePartitions]); + setSplitState(SplitState::kPreAlloc); // Calculate buffer size based on available offheap memory, history average bytes per row and options_.buffer_size. auto preAllocBufferSize = calculatePartitionBufferSize(rv, memLimit); @@ -685,8 +648,9 @@ arrow::Status VeloxShuffleWriter::doSplit(const velox::RowVector& rv, int64_t me return arrow::Status::OK(); } -arrow::Status VeloxShuffleWriter::splitRowVector(const velox::RowVector& rv) { +arrow::Status VeloxShuffleWriter::splitRowVector(const facebook::velox::RowVector& rv) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingSplitRV]); + // now start to split the RowVector RETURN_NOT_OK(splitFixedWidthValueBuffer(rv)); RETURN_NOT_OK(splitValidityBuffer(rv)); @@ -701,7 +665,7 @@ arrow::Status VeloxShuffleWriter::splitRowVector(const velox::RowVector& rv) { return arrow::Status::OK(); } -arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVector& rv) { +arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const facebook::velox::RowVector& rv) { for (auto col = 0; col < fixedWidthColumnCount_; ++col) { auto colIdx = simpleColumnIndices_[col]; auto column = rv.childAt(colIdx); @@ -724,8 +688,8 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); break; case 64: { - if (column->type()->kind() == velox::TypeKind::TIMESTAMP) { - RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); + if (column->type()->kind() == facebook::velox::TypeKind::TIMESTAMP) { + RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); break; } else { #ifdef PROCESSAVX @@ -803,7 +767,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); } else if (column->type()->isLongDecimal()) { // assume batch size = 32k; reducer# = 4K; row/reducer = 8 - RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); + RETURN_NOT_OK(splitFixedType(srcAddr, dstAddrs)); } else { return arrow::Status::Invalid( "Column type " + schema_->field(colIdx)->type()->ToString() + " is not supported."); @@ -908,7 +872,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::splitValidityBuffer(const velox::RowVector& rv) { + arrow::Status VeloxShuffleWriter::splitValidityBuffer(const facebook::velox::RowVector& rv) { for (size_t col = 0; col < simpleColumnIndices_.size(); ++col) { auto colIdx = simpleColumnIndices_[col]; auto column = rv.childAt(colIdx); @@ -916,11 +880,11 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec auto& dstAddrs = partitionValidityAddrs_[col]; for (auto& pid : partitionUsed_) { if (dstAddrs[pid] == nullptr) { - // init bitmap if it's null, initialize the buffer as true - auto newSize = std::max(partition2RowCount_[pid], (uint32_t)options_.buffer_size); - GLUTEN_ASSIGN_OR_THROW( + // Init bitmap if it's null. + ARROW_ASSIGN_OR_RAISE( auto validityBuffer, - arrow::AllocateResizableBuffer(arrow::bit_util::BytesForBits(newSize), partitionBufferPool_.get())); + arrow::AllocateResizableBuffer( + arrow::bit_util::BytesForBits(partition2BufferSize_[pid]), partitionBufferPool_.get())); dstAddrs[pid] = const_cast(validityBuffer->data()); memset(validityBuffer->mutable_data(), 0xff, validityBuffer->capacity()); partitionBuffers_[col][pid][kValidityBufferIndex] = std::move(validityBuffer); @@ -937,14 +901,16 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec } arrow::Status VeloxShuffleWriter::splitBinaryType( - uint32_t binaryIdx, const velox::FlatVector& src, std::vector& dst) { + uint32_t binaryIdx, + const facebook::velox::FlatVector& src, + std::vector& dst) { auto rawValues = src.rawValues(); for (auto& pid : partitionUsed_) { auto& binaryBuf = dst[pid]; // use 32bit offset - auto dstOffsetBase = (BinaryArrayOffsetType*)(binaryBuf.offsetPtr) + partitionBufferIdxBase_[pid]; + auto dstOffsetBase = (BinaryArrayLengthBufferType*)(binaryBuf.offsetPtr) + partitionBufferIdxBase_[pid]; auto valueOffset = binaryBuf.valueOffset; auto dstValuePtr = binaryBuf.valuePtr + valueOffset; @@ -971,7 +937,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec multiply = std::min(3, multiply + 1); auto valueBuffer = std::static_pointer_cast( - partitionBuffers_[fixedWidthColumnCount_ + binaryIdx][pid][kValueBufferIndex]); + partitionBuffers_[fixedWidthColumnCount_ + binaryIdx][pid][kBinaryValueBufferIndex]); RETURN_NOT_OK(valueBuffer->Reserve(capacity)); @@ -992,20 +958,20 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::splitBinaryArray(const velox::RowVector& rv) { + arrow::Status VeloxShuffleWriter::splitBinaryArray(const facebook::velox::RowVector& rv) { for (auto col = fixedWidthColumnCount_; col < simpleColumnIndices_.size(); ++col) { auto binaryIdx = col - fixedWidthColumnCount_; auto& dstAddrs = partitionBinaryAddrs_[binaryIdx]; auto colIdx = simpleColumnIndices_[col]; auto column = rv.childAt(colIdx); - auto stringColumn = column->asFlatVector(); + auto stringColumn = column->asFlatVector(); assert(stringColumn); RETURN_NOT_OK(splitBinaryType(binaryIdx, *stringColumn, dstAddrs)); } return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::splitComplexType(const velox::RowVector& rv) { + arrow::Status VeloxShuffleWriter::splitComplexType(const facebook::velox::RowVector& rv) { if (complexColumnIndices_.size() == 0) { return arrow::Status::OK(); } @@ -1023,17 +989,17 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec complexTypeData_[partition] = serde_.createSerializer( complexWriteType_, partition2RowCount_[partition], arenas_[partition].get(), /* serdeOptions */ nullptr); } - rowIndexs[partition].emplace_back(IndexRange{row, 1}); + rowIndexs[partition].emplace_back(facebook::velox::IndexRange{row, 1}); } - std::vector childrens; + std::vector childrens; for (size_t i = 0; i < complexColumnIndices_.size(); ++i) { auto colIdx = complexColumnIndices_[i]; auto column = rv.childAt(colIdx); childrens.emplace_back(column); } - auto rowVector = std::make_shared( - veloxPool_.get(), complexWriteType_, BufferPtr(nullptr), rv.size(), std::move(childrens)); + auto rowVector = std::make_shared( + veloxPool_.get(), complexWriteType_, facebook::velox::BufferPtr(nullptr), rv.size(), std::move(childrens)); for (auto& pid : partitionUsed_) { if (rowIndexs[pid].size() != 0) { @@ -1044,7 +1010,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::initColumnTypes(const velox::RowVector& rv) { + arrow::Status VeloxShuffleWriter::initColumnTypes(const facebook::velox::RowVector& rv) { schema_ = toArrowSchema(rv.type(), veloxPool_.get()); for (size_t i = 0; i < rv.childrenSize(); ++i) { @@ -1057,7 +1023,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec ARROW_ASSIGN_OR_RAISE(arrowColumnTypes_, toShuffleWriterTypeId(schema_->fields())); std::vector complexNames; - std::vector complexChildrens; + std::vector complexChildrens; for (size_t i = 0; i < arrowColumnTypes_.size(); ++i) { switch (arrowColumnTypes_[i]->id()) { @@ -1091,12 +1057,13 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec complexTypeData_.resize(numPartitions_); complexTypeFlushBuffer_.resize(numPartitions_); - complexWriteType_ = std::make_shared(std::move(complexNames), std::move(complexChildrens)); + complexWriteType_ = + std::make_shared(std::move(complexNames), std::move(complexChildrens)); return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::initFromRowVector(const velox::RowVector& rv) { + arrow::Status VeloxShuffleWriter::initFromRowVector(const facebook::velox::RowVector& rv) { if (veloxColumnTypes_.empty()) { RETURN_NOT_OK(initColumnTypes(rv)); RETURN_NOT_OK(initPartitions()); @@ -1113,14 +1080,15 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec void VeloxShuffleWriter::calculateSimpleColumnBytes() { simpleColumnBytes_ = 0; - for (size_t col = 0; col < simpleColumnIndices_.size(); ++col) { + for (size_t col = 0; col < fixedWidthColumnCount_; ++col) { auto colIdx = simpleColumnIndices_[col]; // `bool(1) >> 3` gets 0, so +7 simpleColumnBytes_ += ((arrow::bit_width(arrowColumnTypes_[colIdx]->id()) + 7) >> 3); } + simpleColumnBytes_ += kSizeOfBinaryArrayLengthBuffer * binaryColumnIndices_.size(); } - uint32_t VeloxShuffleWriter::calculatePartitionBufferSize(const velox::RowVector& rv, int64_t memLimit) { + uint32_t VeloxShuffleWriter::calculatePartitionBufferSize(const facebook::velox::RowVector& rv, int64_t memLimit) { uint32_t bytesPerRow = simpleColumnBytes_; SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingCalculateBufferSize]); @@ -1129,7 +1097,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec std::vector binaryArrayAvgBytesPerRow(binaryColumnIndices_.size()); for (size_t i = 0; i < binaryColumnIndices_.size(); ++i) { auto column = rv.childAt(binaryColumnIndices_[i]); - auto stringViewColumn = column->asFlatVector(); + auto stringViewColumn = column->asFlatVector(); assert(stringViewColumn); uint64_t binarySizeBytes = stringViewColumn->values()->size(); @@ -1148,8 +1116,9 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec memLimit += cachedPayloadSize(); // make sure split buffer uses 128M memory at least, let's hardcode it here for now - if (memLimit < kMinMemLimit) + if (memLimit < kMinMemLimit) { memLimit = kMinMemLimit; + } uint64_t preAllocRowCnt = memLimit > 0 && bytesPerRow > 0 ? memLimit / bytesPerRow / numPartitions_ >> 2 : options_.buffer_size; @@ -1188,86 +1157,48 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::allocatePartitionBuffer(uint32_t partitionId, uint32_t newSize, bool reuseBuffers) { + arrow::Status VeloxShuffleWriter::allocatePartitionBuffer(uint32_t partitionId, uint32_t newSize) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingAllocateBuffer]); - // try to allocate new - auto numFields = schema_->num_fields(); - assert(numFields == arrowColumnTypes_.size()); + for (auto i = 0; i < simpleColumnIndices_.size(); ++i) { + auto columnType = schema_->field(simpleColumnIndices_[i])->type()->id(); + auto& buffers = partitionBuffers_[i][partitionId]; - auto fixedWidthIdx = 0; - auto binaryIdx = 0; - for (auto i = 0; i < numFields; ++i) { - switch (arrowColumnTypes_[i]->id()) { + std::shared_ptr validityBuffer{}; + ARROW_ASSIGN_OR_RAISE(validityBuffer, allocateValidityBuffer(i, partitionId, newSize)); + switch (columnType) { + // binary types case arrow::BinaryType::type_id: case arrow::StringType::type_id: { - std::shared_ptr validityBuffer{}; - std::shared_ptr lengthBuffer{}; - std::shared_ptr valueBuffer{}; + auto binaryIdx = i - fixedWidthColumnCount_; - auto columnIdx = fixedWidthColumnCount_ + binaryIdx; - ARROW_ASSIGN_OR_RAISE(validityBuffer, allocateValidityBuffer(columnIdx, partitionId, newSize)); + std::shared_ptr lengthBuffer{}; + auto lengthBufferSize = newSize * kSizeOfBinaryArrayLengthBuffer; + ARROW_ASSIGN_OR_RAISE( + lengthBuffer, arrow::AllocateResizableBuffer(lengthBufferSize, partitionBufferPool_.get())); - auto valueBufferSize = calculateValueBufferSizeForBinaryArray(binaryIdx, newSize); - auto lengthBufferSize = newSize * sizeof(BinaryArrayOffsetType); + std::shared_ptr valueBuffer{}; + auto valueBufferSize = valueBufferSizeForBinaryArray(binaryIdx, newSize); + ARROW_ASSIGN_OR_RAISE( + valueBuffer, arrow::AllocateResizableBuffer(valueBufferSize, partitionBufferPool_.get())); - auto& buffers = partitionBuffers_[columnIdx][partitionId]; - if (reuseBuffers) { - valueBuffer = std::dynamic_pointer_cast(buffers[kValueBufferIndex]); - RETURN_NOT_OK(valueBuffer->Resize(valueBufferSize, /*shrink_to_fit=*/true)); - lengthBuffer = std::dynamic_pointer_cast(buffers[kLengthBufferIndex]); - RETURN_NOT_OK(lengthBuffer->Resize(lengthBufferSize, /*shrink_to_fit=*/true)); - } else { - ARROW_ASSIGN_OR_RAISE( - valueBuffer, arrow::AllocateResizableBuffer(valueBufferSize, partitionBufferPool_.get())); - ARROW_ASSIGN_OR_RAISE( - lengthBuffer, arrow::AllocateResizableBuffer(lengthBufferSize, partitionBufferPool_.get())); - } partitionBinaryAddrs_[binaryIdx][partitionId] = BinaryBuf(valueBuffer->mutable_data(), lengthBuffer->mutable_data(), valueBufferSize); buffers = {std::move(validityBuffer), std::move(lengthBuffer), std::move(valueBuffer)}; - - binaryIdx++; break; } - case arrow::StructType::type_id: - case arrow::MapType::type_id: - case arrow::ListType::type_id: - break; - default: { - std::shared_ptr validityBuffer{}; + default: { // fixed-width types std::shared_ptr valueBuffer{}; - - ARROW_ASSIGN_OR_RAISE(validityBuffer, allocateValidityBuffer(fixedWidthIdx, partitionId, newSize)); - - int64_t valueBufferSize = 0; - if (arrowColumnTypes_[i]->id() == arrow::BooleanType::type_id) { - valueBufferSize = arrow::bit_util::BytesForBits(newSize); - } else if (veloxColumnTypes_[i]->isShortDecimal()) { - valueBufferSize = newSize * (arrow::bit_width(arrow::Int64Type::type_id) >> 3); - } else if (veloxColumnTypes_[i]->kind() == TypeKind::TIMESTAMP) { - valueBufferSize = BaseVector::byteSize(newSize); - } else { - valueBufferSize = newSize * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3); - } - - auto& buffers = partitionBuffers_[fixedWidthIdx][partitionId]; - if (reuseBuffers) { - valueBuffer = std::dynamic_pointer_cast(buffers[1]); - RETURN_NOT_OK(valueBuffer->Resize(valueBufferSize, /*shrink_to_fit=*/true)); - } else { - ARROW_ASSIGN_OR_RAISE( - valueBuffer, arrow::AllocateResizableBuffer(valueBufferSize, partitionBufferPool_.get())); - } - partitionFixedWidthValueAddrs_[fixedWidthIdx][partitionId] = valueBuffer->mutable_data(); + ARROW_ASSIGN_OR_RAISE( + valueBuffer, + arrow::AllocateResizableBuffer( + valueBufferSizeForFixedWidthArray(i, newSize), partitionBufferPool_.get())); + partitionFixedWidthValueAddrs_[i][partitionId] = valueBuffer->mutable_data(); buffers = {std::move(validityBuffer), std::move(valueBuffer)}; - - fixedWidthIdx++; break; } } } - partition2BufferSize_[partitionId] = newSize; return arrow::Status::OK(); } @@ -1276,17 +1207,17 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec uint32_t partitionId, bool reuseBuffers) { ARROW_ASSIGN_OR_RAISE(auto rb, createArrowRecordBatchFromBuffer(partitionId, reuseBuffers)); if (rb) { - ARROW_ASSIGN_OR_RAISE(auto payload, createArrowIpcPayload(*rb, reuseBuffers)); + ARROW_ASSIGN_OR_RAISE(auto payload, createPayload(*rb, reuseBuffers)); return payload; } return nullptr; } - arrow::Result> VeloxShuffleWriter::createArrowIpcPayload( + arrow::Result> VeloxShuffleWriter::createPayload( const arrow::RecordBatch& rb, bool reuseBuffers) { auto payload = std::make_unique(); // Extract numRows from header column - GLUTEN_THROW_NOT_OK(arrow::ipc::GetRecordBatchPayload(rb, options_.ipc_write_options, payload.get())); + RETURN_NOT_OK(arrow::ipc::GetRecordBatchPayload(rb, options_.ipc_write_options, payload.get())); if (codec_ == nullptr) { // Without compression, we need to perform a manual copy of the original buffers // so that we can reuse them for next split. @@ -1310,7 +1241,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec uint32_t partitionId, bool reuseBuffers) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingCreateRbFromBuffer]); - if (partitionBufferIdxBase_[partitionId] <= 0) { + if (partitionBufferIdxBase_[partitionId] == 0) { return nullptr; } @@ -1341,11 +1272,13 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec } // offset buffer ARROW_RETURN_IF( - !buffers[kLengthBufferIndex], arrow::Status::Invalid("Offset buffer of binary array is null.")); - allBuffers.push_back(arrow::SliceBuffer(buffers[kLengthBufferIndex], 0, numRows * sizeof(int32_t))); - ARROW_RETURN_IF(!buffers[kValueBufferIndex], arrow::Status::Invalid("Value buffer of binary array is null.")); + !buffers[kBinaryLengthBufferIndex], arrow::Status::Invalid("Offset buffer of binary array is null.")); + allBuffers.push_back( + arrow::SliceBuffer(buffers[kBinaryLengthBufferIndex], 0, numRows * kSizeOfBinaryArrayLengthBuffer)); + ARROW_RETURN_IF( + !buffers[kBinaryValueBufferIndex], arrow::Status::Invalid("Value buffer of binary array is null.")); // value buffer - allBuffers.push_back(arrow::SliceBuffer(buffers[kValueBufferIndex], 0, binaryBuf.valueOffset)); + allBuffers.push_back(arrow::SliceBuffer(buffers[kBinaryValueBufferIndex], 0, binaryBuf.valueOffset)); if (reuseBuffers) { // Set the first value offset to 0. @@ -1374,20 +1307,22 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec allBuffers.push_back(nullptr); } // value buffer - ARROW_RETURN_IF(!buffers[1], arrow::Status::Invalid("Value buffer of fixed-width array is null.")); - std::shared_ptr valueBuffer; + auto& valueBuffer = buffers[kFixedWidthValueBufferIndex]; + ARROW_RETURN_IF(!valueBuffer, arrow::Status::Invalid("Value buffer of fixed-width array is null.")); + std::shared_ptr slicedValueBuffer; if (arrowColumnTypes_[i]->id() == arrow::BooleanType::type_id) { - valueBuffer = arrow::SliceBuffer(buffers[1], 0, arrow::bit_util::BytesForBits(numRows)); + slicedValueBuffer = arrow::SliceBuffer(valueBuffer, 0, arrow::bit_util::BytesForBits(numRows)); } else if (veloxColumnTypes_[i]->isShortDecimal()) { - valueBuffer = - arrow::SliceBuffer(buffers[1], 0, numRows * (arrow::bit_width(arrow::Int64Type::type_id) >> 3)); - } else if (veloxColumnTypes_[i]->kind() == TypeKind::TIMESTAMP) { - valueBuffer = arrow::SliceBuffer(buffers[1], 0, BaseVector::byteSize(numRows)); + slicedValueBuffer = + arrow::SliceBuffer(valueBuffer, 0, numRows * (arrow::bit_width(arrow::Int64Type::type_id) >> 3)); + } else if (veloxColumnTypes_[i]->kind() == facebook::velox::TypeKind::TIMESTAMP) { + slicedValueBuffer = arrow::SliceBuffer( + valueBuffer, 0, facebook::velox::BaseVector::byteSize(numRows)); } else { - valueBuffer = - arrow::SliceBuffer(buffers[1], 0, numRows * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3)); + slicedValueBuffer = + arrow::SliceBuffer(valueBuffer, 0, numRows * (arrow::bit_width(arrowColumnTypes_[i]->id()) >> 3)); } - allBuffers.push_back(std::move(valueBuffer)); + allBuffers.push_back(std::move(slicedValueBuffer)); if (!reuseBuffers) { partitionValidityAddrs_[fixedWidthIdx][partitionId] = nullptr; partitionFixedWidthValueAddrs_[fixedWidthIdx][partitionId] = nullptr; @@ -1402,13 +1337,13 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec auto flushBuffer = complexTypeFlushBuffer_[partitionId]; auto serializedSize = complexTypeData_[partitionId]->maxSerializedSize(); if (flushBuffer == nullptr) { - GLUTEN_ASSIGN_OR_THROW(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); + ARROW_ASSIGN_OR_RAISE(flushBuffer, arrow::AllocateResizableBuffer(serializedSize, payloadPool_.get())); } else if (serializedSize > flushBuffer->capacity()) { - GLUTEN_THROW_NOT_OK(flushBuffer->Reserve(serializedSize)); + RETURN_NOT_OK(flushBuffer->Reserve(serializedSize)); } auto valueBuffer = arrow::SliceMutableBuffer(flushBuffer, 0, serializedSize); auto output = std::make_shared(valueBuffer); - serializer::presto::PrestoOutputStreamListener listener; + facebook::velox::serializer::presto::PrestoOutputStreamListener listener; ArrowFixedSizeBufferOutputStream out(output, &listener); complexTypeData_[partitionId]->flush(&out); allBuffers.emplace_back(valueBuffer); @@ -1425,80 +1360,72 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return makeRecordBatch(numRows, allBuffers); } - std::shared_ptr VeloxShuffleWriter::makeRecordBatch( + arrow::Result> VeloxShuffleWriter::makeRecordBatch( uint32_t numRows, const std::vector>& buffers) { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingMakeRB]); if (codec_ == nullptr) { return makeUncompressedRecordBatch(numRows, buffers, writeSchema(), payloadPool_.get()); } else { - TIME_NANO_START(totalCompressTime_); - auto rb = makeCompressedRecordBatch( + return makeCompressedRecordBatch( numRows, buffers, compressWriteSchema(), payloadPool_.get(), codec_.get(), - options_.buffer_compress_threshold, - options_.compression_mode); - TIME_NANO_END(totalCompressTime_); - return rb; + options_.compression_threshold, + options_.compression_mode, + totalCompressTime_); } } arrow::Status VeloxShuffleWriter::evictFixedSize(int64_t size, int64_t * actual) { - int64_t currentEvicted = 0; + if (evictState_ == EvictState::kUnevictable) { + *actual = 0; + return arrow::Status::OK(); + } + EvictGuard evictGuard{evictState_}; - // If OOM happens during stop(), the reclaim order is shrink->spill, - // because the partition buffers will be freed soon. - if (currentEvicted < size && shrinkBeforeSpill()) { + int64_t reclaimed = 0; + if (reclaimed < size && shrinkPartitionBuffersBeforeSpill()) { ARROW_ASSIGN_OR_RAISE(auto shrunken, shrinkPartitionBuffers()); - currentEvicted += shrunken; + reclaimed += shrunken; } - - auto tryCount = 0; - while (currentEvicted < size && tryCount < 5) { - tryCount++; - int64_t singleCallEvicted = 0; - RETURN_NOT_OK(evictPartitionsOnDemand(&singleCallEvicted)); - if (singleCallEvicted <= 0) { - break; - } - currentEvicted += singleCallEvicted; + if (reclaimed < size) { + ARROW_ASSIGN_OR_RAISE(auto cached, evictCachedPayload()); + reclaimed += cached; } - - // If OOM happens during binary buffers resize, the reclaim order is spill->shrink, - // because the partition buffers can be reused. - if (currentEvicted < size && shrinkAfterSpill()) { - ARROW_ASSIGN_OR_RAISE(auto shrunken, shrinkPartitionBuffers()); - currentEvicted += shrunken; + if (reclaimed < size && shrinkPartitionBuffersAfterSpill()) { + ARROW_ASSIGN_OR_RAISE(auto shrunken, shrinkPartitionBuffersMinSize(size - reclaimed)); + reclaimed += shrunken; } - - *actual = currentEvicted; + if (reclaimed < size && evictPartitionBuffersAfterSpill()) { + ARROW_ASSIGN_OR_RAISE(auto evicted, evictPartitionBuffersMinSize(size - reclaimed)); + reclaimed += evicted; + } + *actual = reclaimed; return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::evictPartitionsOnDemand(int64_t * size) { + arrow::Result VeloxShuffleWriter::evictCachedPayload() { SCOPED_TIMER(cpuWallTimingList_[CpuWallTimingEvictPartition]); - // Evict all cached partitions auto beforeEvict = cachedPayloadSize(); if (beforeEvict == 0) { - *size = 0; - } else { - RETURN_NOT_OK(partitionWriter_->spill()); - if (auto afterEvict = cachedPayloadSize()) { - if (splitState_ != SplitState::kPreAlloc && splitState_ != SplitState::kStop) { - // Apart from kPreAlloc and kStop states, spill should not be triggered by allocating payload buffers. All - // cached data should be evicted. - return arrow::Status::Invalid( - "Not all cached payload evicted." + std::to_string(afterEvict) + " bytes remains."); - } - *size = beforeEvict - afterEvict; - } else { - DLOG(INFO) << "Evicted all partitions. " << std::to_string(beforeEvict) << " bytes released" << std::endl; - *size = beforeEvict; - } + return 0; } - return arrow::Status::OK(); + auto evicted = beforeEvict; + + { + ScopedTimer evictTime(totalEvictTime_); + RETURN_NOT_OK(partitionWriter_->finishEvict()); + } + + if (auto afterEvict = cachedPayloadSize()) { + // Evict can be triggered by compressing buffers. The cachedPayloadSize is not empty. + evicted -= afterEvict; + } + + DLOG(INFO) << "Evicted all cached payloads. " << std::to_string(evicted) << " bytes released" << std::endl; + return evicted; } arrow::Status VeloxShuffleWriter::resetValidityBuffer(uint32_t partitionId) { @@ -1512,14 +1439,17 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - arrow::Status VeloxShuffleWriter::resizePartitionBuffer(uint32_t partitionId, int64_t newSize) { + arrow::Status VeloxShuffleWriter::resizePartitionBuffer(uint32_t partitionId, int64_t newSize, bool preserveData) { for (auto i = 0; i < simpleColumnIndices_.size(); ++i) { auto columnType = schema_->field(simpleColumnIndices_[i])->type()->id(); auto& buffers = partitionBuffers_[i][partitionId]; - // resize validity - if (buffers[kValidityBufferIndex]) { - auto& validityBuffer = buffers[kValidityBufferIndex]; + // Handle validity buffer first. + auto& validityBuffer = buffers[kValidityBufferIndex]; + if (!preserveData) { + ARROW_ASSIGN_OR_RAISE(validityBuffer, allocateValidityBuffer(i, partitionId, newSize)); + } else if (buffers[kValidityBufferIndex]) { + // Resize validity. auto filled = validityBuffer->capacity(); RETURN_NOT_OK(validityBuffer->Resize(arrow::bit_util::BytesForBits(newSize))); partitionValidityAddrs_[i][partitionId] = validityBuffer->mutable_data(); @@ -1530,42 +1460,39 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec } } - // shrink value buffer if fixed-width, offset & value buffers if binary + // Resize value buffer if fixed-width, offset & value buffers if binary. switch (columnType) { // binary types case arrow::BinaryType::type_id: case arrow::StringType::type_id: { - auto& lengthBuffer = buffers[kLengthBufferIndex]; + // Resize length buffer. + auto& lengthBuffer = buffers[kBinaryLengthBufferIndex]; ARROW_RETURN_IF(!lengthBuffer, arrow::Status::Invalid("Offset buffer of binary array is null.")); - RETURN_NOT_OK(lengthBuffer->Resize(newSize * sizeof(BinaryArrayOffsetType))); + RETURN_NOT_OK(lengthBuffer->Resize(newSize * kSizeOfBinaryArrayLengthBuffer)); + // Resize value buffer. auto binaryIdx = i - fixedWidthColumnCount_; auto& binaryBuf = partitionBinaryAddrs_[binaryIdx][partitionId]; - auto& valueBuffer = buffers[kValueBufferIndex]; + auto& valueBuffer = buffers[kBinaryValueBufferIndex]; ARROW_RETURN_IF(!valueBuffer, arrow::Status::Invalid("Value buffer of binary array is null.")); - auto binaryNewSize = calculateValueBufferSizeForBinaryArray(binaryIdx, newSize); - auto valueBufferSize = std::max(binaryBuf.valueOffset, binaryNewSize); + // Determine the new Size for value buffer. + auto valueBufferSize = valueBufferSizeForBinaryArray(binaryIdx, newSize); + auto valueOffset = 0; + // If preserve data, the new valueBufferSize should not be smaller than the current offset. + if (preserveData) { + valueBufferSize = std::max(binaryBuf.valueOffset, valueBufferSize); + valueOffset = binaryBuf.valueOffset; + } RETURN_NOT_OK(valueBuffer->Resize(valueBufferSize)); - binaryBuf = BinaryBuf( - valueBuffer->mutable_data(), lengthBuffer->mutable_data(), valueBufferSize, binaryBuf.valueOffset); + binaryBuf = + BinaryBuf(valueBuffer->mutable_data(), lengthBuffer->mutable_data(), valueBufferSize, valueOffset); break; } default: { // fixed-width types - uint64_t valueBufferSize = 0; - auto columnIndex = simpleColumnIndices_[i]; - if (arrowColumnTypes_[columnIndex]->id() == arrow::BooleanType::type_id) { - valueBufferSize = arrow::bit_util::BytesForBits(newSize); - } else if (veloxColumnTypes_[columnIndex]->isShortDecimal()) { - valueBufferSize = newSize * (arrow::bit_width(arrow::Int64Type::type_id) >> 3); - } else if (veloxColumnTypes_[columnIndex]->kind() == TypeKind::TIMESTAMP) { - valueBufferSize = BaseVector::byteSize(newSize); - } else { - valueBufferSize = newSize * (arrow::bit_width(arrowColumnTypes_[columnIndex]->id()) >> 3); - } - auto& valueBuffer = buffers[1]; + auto& valueBuffer = buffers[kFixedWidthValueBufferIndex]; ARROW_RETURN_IF(!valueBuffer, arrow::Status::Invalid("Value buffer of fixed-width array is null.")); - RETURN_NOT_OK(valueBuffer->Resize(valueBufferSize)); + RETURN_NOT_OK(valueBuffer->Resize(valueBufferSizeForFixedWidthArray(i, newSize))); partitionFixedWidthValueAddrs_[i][partitionId] = valueBuffer->mutable_data(); break; } @@ -1581,7 +1508,7 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::OK(); } - ARROW_ASSIGN_OR_RAISE(auto newSize, sizeAfterShrink(partitionId)); + ARROW_ASSIGN_OR_RAISE(auto newSize, partitionBufferSizeAfterShrink(partitionId)); if (newSize > bufferSize) { std::stringstream invalid; invalid << "Cannot shrink to larger size. Partition: " << partitionId << ", before shrink: " << bufferSize @@ -1589,12 +1516,13 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec return arrow::Status::Invalid(invalid.str()); } if (newSize == bufferSize) { + // No space to shrink. return arrow::Status::OK(); } if (newSize == 0) { return resetPartitionBuffer(partitionId); } - return resizePartitionBuffer(partitionId, newSize); + return resizePartitionBuffer(partitionId, newSize, /*preserveData=*/true); } arrow::Result VeloxShuffleWriter::shrinkPartitionBuffers() { @@ -1606,14 +1534,29 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec RETURN_NOT_OK(shrinkPartitionBuffer(pid)); } auto shrunken = beforeShrink - partitionBufferPool_->bytes_allocated(); - DLOG(INFO) << shrunken << " bytes released from shrinking."; + VLOG(2) << shrunken << " bytes released from shrinking."; return shrunken; } - uint64_t VeloxShuffleWriter::calculateValueBufferSizeForBinaryArray(uint32_t binaryIdx, int64_t newSize) { + uint64_t VeloxShuffleWriter::valueBufferSizeForBinaryArray(uint32_t binaryIdx, int64_t newSize) { return (binaryArrayTotalSizeBytes_[binaryIdx] + totalInputNumRows_ - 1) / totalInputNumRows_ * newSize + 1024; } + uint64_t VeloxShuffleWriter::valueBufferSizeForFixedWidthArray(uint32_t fixedWidthIdx, int64_t newSize) { + uint64_t valueBufferSize = 0; + auto columnIdx = simpleColumnIndices_[fixedWidthIdx]; + if (arrowColumnTypes_[columnIdx]->id() == arrow::BooleanType::type_id) { + valueBufferSize = arrow::bit_util::BytesForBits(newSize); + } else if (veloxColumnTypes_[columnIdx]->isShortDecimal()) { + valueBufferSize = newSize * (arrow::bit_width(arrow::Int64Type::type_id) >> 3); + } else if (veloxColumnTypes_[columnIdx]->kind() == facebook::velox::TypeKind::TIMESTAMP) { + valueBufferSize = facebook::velox::BaseVector::byteSize(newSize); + } else { + valueBufferSize = newSize * (arrow::bit_width(arrowColumnTypes_[columnIdx]->id()) >> 3); + } + return valueBufferSize; + } + void VeloxShuffleWriter::stat() const { #if VELOX_SHUFFLE_WRITER_LOG_FLAG for (int i = CpuWallTimingBegin; i != CpuWallTimingEnd; ++i) { @@ -1664,19 +1607,105 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec arrow::Status VeloxShuffleWriter::evictPayload( uint32_t partitionId, std::unique_ptr payload) { - return partitionWriter_->processPayload(partitionId, std::move(payload)); + ScopedTimer spillTime(totalEvictTime_); + if (!partitionWriter_->getEvictHandle()) { + RETURN_NOT_OK(partitionWriter_->requestNextEvict(false)); + } + RETURN_NOT_OK(partitionWriter_->getEvictHandle()->evict(partitionId, std::move(payload))); + return arrow::Status::OK(); } - bool VeloxShuffleWriter::shrinkBeforeSpill() const { - return options_.partitioning_name != "single" && splitState_ == SplitState::kStop; + arrow::Result VeloxShuffleWriter::shrinkPartitionBuffersMinSize(int64_t size) { + // Sort partition buffers by (partition2BufferSize_ - partitionBufferIdxBase_) + std::vector> pidToSize; + for (auto pid = 0; pid < numPartitions_; ++pid) { + if (partition2BufferSize_[pid] > 0 && partition2BufferSize_[pid] > partitionBufferIdxBase_[pid]) { + pidToSize.emplace_back(pid, partition2BufferSize_[pid] - partitionBufferIdxBase_[pid]); + } + } + // No shrinkable partition buffer. + if (pidToSize.empty()) { + return 0; + } + + std::sort(pidToSize.begin(), pidToSize.end(), [&](const auto& a, const auto& b) { return a.second > b.second; }); + + auto beforeShrink = partitionBufferPool_->bytes_allocated(); + auto shrunken = 0; + auto iter = pidToSize.begin(); + + // Shrink in order to reclaim the largest amount of space with fewer resizes. + do { + RETURN_NOT_OK(shrinkPartitionBuffer(iter->first)); + shrunken = beforeShrink - partitionBufferPool_->bytes_allocated(); + iter++; + } while (shrunken < size && iter != pidToSize.end()); + return shrunken; } - bool VeloxShuffleWriter::shrinkAfterSpill() const { - return options_.partitioning_name != "single" && + arrow::Result VeloxShuffleWriter::evictPartitionBuffersMinSize(int64_t size) { + // Evict partition buffers, only when splitState_ == SplitState::kInit, and space freed from + // shrinking is not enough. In this case partition2BufferSize_ == partitionBufferIdxBase_ + int64_t beforeEvict = partitionBufferPool_->bytes_allocated(); + int64_t evicted = 0; + std::vector> pidToSize; + for (auto pid = 0; pid < numPartitions_; ++pid) { + if (partition2BufferSize_[pid] == 0) { + continue; + } + pidToSize.emplace_back(pid, partition2BufferSize_[pid]); + } + if (!pidToSize.empty()) { + Timer spillTime; + spillTime.start(); + RETURN_NOT_OK(partitionWriter_->requestNextEvict(true)); + auto evictHandle = partitionWriter_->getEvictHandle(); + spillTime.stop(); + + for (auto& item : pidToSize) { + auto pid = item.first; + ARROW_ASSIGN_OR_RAISE(auto payload, createPayloadFromBuffer(pid, false)); + + spillTime.start(); + RETURN_NOT_OK(evictHandle->evict(pid, std::move(payload))); + spillTime.stop(); + + evicted = beforeEvict - partitionBufferPool_->bytes_allocated(); + if (evicted >= size) { + break; + } + } + spillTime.start(); + RETURN_NOT_OK(partitionWriter_->finishEvict()); + spillTime.stop(); + totalEvictTime_ += spillTime.realTimeUsed(); + } + return evicted; + } + + bool VeloxShuffleWriter::shrinkPartitionBuffersBeforeSpill() const { + // If OOM happens during stop(), the reclaim order is shrink->spill, + // because the partition buffers will be freed soon. + // SinglePartitioning doesn't maintain partition buffers. + return options_.partitioning != Partitioning::kSingle && splitState_ == SplitState::kStop; + } + + bool VeloxShuffleWriter::shrinkPartitionBuffersAfterSpill() const { + // If OOM happens during SplitState::kSplit, it is triggered by binary buffers resize. + // Or during SplitState::kInit, it is triggered by other operators. + // The reclaim order is spill->shrink, because the partition buffers can be reused. + // SinglePartitioning doesn't maintain partition buffers. + return options_.partitioning != Partitioning::kSingle && (splitState_ == SplitState::kSplit || splitState_ == SplitState::kInit); } - arrow::Result VeloxShuffleWriter::sizeAfterShrink(uint32_t partitionId) const { + bool VeloxShuffleWriter::evictPartitionBuffersAfterSpill() const { + // If OOM triggered by other operators, the splitState_ is SplitState::kInit. + // The last resort is to evict the partition buffers to reclaim more space. + return options_.partitioning != Partitioning::kSingle && splitState_ == SplitState::kInit; + } + + arrow::Result VeloxShuffleWriter::partitionBufferSizeAfterShrink(uint32_t partitionId) const { if (splitState_ == SplitState::kSplit) { return partitionBufferIdxBase_[partitionId] + partition2RowCount_[partitionId]; } @@ -1692,17 +1721,17 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec // Make sure the size to be allocated is larger than the size to be filled. if (partition2BufferSize_[pid] == 0) { // Allocate buffer if it's not yet allocated. - RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize, false)); + RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize)); } else if (beyondThreshold(pid, newSize)) { if (newSize <= partitionBufferIdxBase_[pid]) { // If the newSize is smaller, cache the buffered data and reuse and shrink the buffer. RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, true)); - RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize, true)); + RETURN_NOT_OK(resizePartitionBuffer(pid, newSize, /*preserveData=*/false)); } else { // If the newSize is larger, check if alreadyFilled + toBeFilled <= newSize if (partitionBufferIdxBase_[pid] + partition2RowCount_[pid] <= newSize) { // If so, keep the data in buffers and resize buffers. - RETURN_NOT_OK(resizePartitionBuffer(pid, newSize)); // resize + RETURN_NOT_OK(resizePartitionBuffer(pid, newSize, /*preserveData=*/true)); // Because inputHasNull_ is updated every time split is called, and resizePartitionBuffer won't allocate // validity buffer. RETURN_NOT_OK(updateValidityBuffers(pid, newSize)); @@ -1712,7 +1741,11 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec // Else free and allocate new buffers. bool reuseBuffers = newSize <= partition2BufferSize_[pid]; RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, reuseBuffers)); - RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize, reuseBuffers)); + if (reuseBuffers) { + RETURN_NOT_OK(resizePartitionBuffer(pid, newSize, /*preserveData=*/false)); + } else { + RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize)); + } } } } else if (partitionBufferIdxBase_[pid] + partition2RowCount_[pid] > partition2BufferSize_[pid]) { @@ -1721,11 +1754,11 @@ arrow::Status VeloxShuffleWriter::splitFixedWidthValueBuffer(const velox::RowVec if (newSize > partition2BufferSize_[pid]) { // If the partition size after split is already larger than allocated buffer size, need reallocate. RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, false)); - RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize, false)); + RETURN_NOT_OK(allocatePartitionBuffer(pid, newSize)); } else { // Partition size after split is smaller than buffer size. Reuse the buffers. RETURN_NOT_OK(evictPartitionBuffer(pid, newSize, true)); - // Reset validity buffer for reuse. + // Reset validity buffer for reallocate. RETURN_NOT_OK(resetValidityBuffer(pid)); } } diff --git a/cpp/velox/shuffle/VeloxShuffleWriter.h b/cpp/velox/shuffle/VeloxShuffleWriter.h index 43f161aaf562..b6155e930600 100644 --- a/cpp/velox/shuffle/VeloxShuffleWriter.h +++ b/cpp/velox/shuffle/VeloxShuffleWriter.h @@ -29,20 +29,12 @@ #include "velox/vector/FlatVector.h" #include "velox/vector/VectorStream.h" -#include -#include -#include +#include #include #include #include +#include #include -#include -#include -#include -#include "arrow/array/builder_base.h" - -#include "arrow/array/util.h" -#include "arrow/result.h" #include "memory/VeloxMemoryManager.h" #include "shuffle/PartitionWriterCreator.h" @@ -96,9 +88,15 @@ namespace gluten { #endif // end of VELOX_SHUFFLE_WRITER_PRINT enum SplitState { kInit, kPreAlloc, kSplit, kStop }; +enum EvictState { kEvictable, kUnevictable }; class VeloxShuffleWriter final : public ShuffleWriter { - enum { kValidityBufferIndex = 0, kLengthBufferIndex = 1, kValueBufferIndex = 2 }; + enum { + kValidityBufferIndex = 0, + kFixedWidthValueBufferIndex = 1, + kBinaryValueBufferIndex = 2, + kBinaryLengthBufferIndex = kFixedWidthValueBufferIndex + }; public: struct BinaryBuf { @@ -191,16 +189,6 @@ class VeloxShuffleWriter final : public ShuffleWriter { VS_PRINT_CONTAINER(input_has_null_); } - // Public for test only. - void setSplitState(SplitState state) { - splitState_ = state; - } - - // For test only. - SplitState getSplitState() { - return splitState_; - } - protected: VeloxShuffleWriter( uint32_t numPartitions, @@ -229,6 +217,8 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Status updateInputHasNull(const facebook::velox::RowVector& rv); + void setSplitState(SplitState state); + arrow::Status doSplit(const facebook::velox::RowVector& rv, int64_t memLimit); bool beyondThreshold(uint32_t partitionId, uint64_t newSize); @@ -242,7 +232,7 @@ class VeloxShuffleWriter final : public ShuffleWriter { arrow::Result> allocateValidityBuffer(uint32_t col, uint32_t partitionId, uint32_t newSize); - arrow::Status allocatePartitionBuffer(uint32_t partitionId, uint32_t newSize, bool reuseBuffers); + arrow::Status allocatePartitionBuffer(uint32_t partitionId, uint32_t newSize); arrow::Status splitFixedWidthValueBuffer(const facebook::velox::RowVector& rv); @@ -260,9 +250,7 @@ class VeloxShuffleWriter final : public ShuffleWriter { uint32_t partitionId, bool reuseBuffers); - arrow::Result> createArrowIpcPayload( - const arrow::RecordBatch& rb, - bool reuseBuffers); + arrow::Result> createPayload(const arrow::RecordBatch& rb, bool reuseBuffers); template arrow::Status splitFixedType(const uint8_t* srcAddr, const std::vector& dstAddrs) { @@ -283,35 +271,46 @@ class VeloxShuffleWriter final : public ShuffleWriter { const facebook::velox::FlatVector& src, std::vector& dst); - arrow::Status evictPartitionsOnDemand(int64_t* size); + arrow::Result evictCachedPayload(); - std::shared_ptr makeRecordBatch( + arrow::Result> makeRecordBatch( uint32_t numRows, const std::vector>& buffers); - std::shared_ptr generateComplexTypeBuffers(facebook::velox::RowVectorPtr vector); + arrow::Result> generateComplexTypeBuffers(facebook::velox::RowVectorPtr vector); arrow::Status resetValidityBuffer(uint32_t partitionId); + arrow::Result shrinkPartitionBuffersMinSize(int64_t size); + arrow::Result shrinkPartitionBuffers(); - arrow::Status resetPartitionBuffer(uint32_t partitionId); + arrow::Result evictPartitionBuffersMinSize(int64_t size); arrow::Status shrinkPartitionBuffer(uint32_t partitionId); - arrow::Status resizePartitionBuffer(uint32_t partitionId, int64_t newSize); + arrow::Status resetPartitionBuffer(uint32_t partitionId); + + // Resize the partition buffer to newSize. If preserveData is true, it will keep the data in buffer. + // Note when preserveData is false, and newSize is larger, this function can introduce unnecessary memory copy. + // In this case, use allocatePartitionBuffer to free current buffers and allocate new buffers instead. + arrow::Status resizePartitionBuffer(uint32_t partitionId, int64_t newSize, bool preserveData); - uint64_t calculateValueBufferSizeForBinaryArray(uint32_t binaryIdx, int64_t newSize); + uint64_t valueBufferSizeForBinaryArray(uint32_t binaryIdx, int64_t newSize); + + uint64_t valueBufferSizeForFixedWidthArray(uint32_t fixedWidthIndex, int64_t newSize); void calculateSimpleColumnBytes(); void stat() const; - bool shrinkBeforeSpill() const; + bool shrinkPartitionBuffersBeforeSpill() const; + + bool shrinkPartitionBuffersAfterSpill() const; - bool shrinkAfterSpill() const; + bool evictPartitionBuffersAfterSpill() const; - arrow::Result sizeAfterShrink(uint32_t partitionId) const; + arrow::Result partitionBufferSizeAfterShrink(uint32_t partitionId) const; protected: // Memory Pool used to track memory allocation of Arrow IPC payloads. @@ -322,6 +321,8 @@ class VeloxShuffleWriter final : public ShuffleWriter { SplitState splitState_{kInit}; + EvictState evictState_{kEvictable}; + bool supportAvx512_ = false; // store arrow column types diff --git a/cpp/velox/substrait/SubstraitParser.cc b/cpp/velox/substrait/SubstraitParser.cc index b7f60ec21f4c..b7e45b9f8b70 100644 --- a/cpp/velox/substrait/SubstraitParser.cc +++ b/cpp/velox/substrait/SubstraitParser.cc @@ -19,169 +19,87 @@ #include "TypeUtils.h" #include "velox/common/base/Exceptions.h" +#include "VeloxSubstraitSignature.h" + namespace gluten { -SubstraitParser::SubstraitType SubstraitParser::parseType(const ::substrait::Type& substraitType) { - // The used type names should be aligned with those in Velox. - std::string typeName; - ::substrait::Type_Nullability nullability; +TypePtr SubstraitParser::parseType(const ::substrait::Type& substraitType, bool asLowerCase) { switch (substraitType.kind_case()) { - case ::substrait::Type::KindCase::kBool: { - typeName = "BOOLEAN"; - nullability = substraitType.bool_().nullability(); - break; - } - case ::substrait::Type::KindCase::kI8: { - typeName = "TINYINT"; - nullability = substraitType.i8().nullability(); - break; - } - case ::substrait::Type::KindCase::kI16: { - typeName = "SMALLINT"; - nullability = substraitType.i16().nullability(); - break; - } - case ::substrait::Type::KindCase::kI32: { - typeName = "INTEGER"; - nullability = substraitType.i32().nullability(); - break; - } - case ::substrait::Type::KindCase::kI64: { - typeName = "BIGINT"; - nullability = substraitType.i64().nullability(); - break; - } - case ::substrait::Type::KindCase::kFp32: { - typeName = "REAL"; - nullability = substraitType.fp32().nullability(); - break; - } - case ::substrait::Type::KindCase::kFp64: { - typeName = "DOUBLE"; - nullability = substraitType.fp64().nullability(); - break; - } - case ::substrait::Type::KindCase::kString: { - typeName = "VARCHAR"; - nullability = substraitType.string().nullability(); - break; - } - case ::substrait::Type::KindCase::kBinary: { - typeName = "VARBINARY"; - nullability = substraitType.string().nullability(); - break; - } + case ::substrait::Type::KindCase::kBool: + return BOOLEAN(); + case ::substrait::Type::KindCase::kI8: + return TINYINT(); + case ::substrait::Type::KindCase::kI16: + return SMALLINT(); + case ::substrait::Type::KindCase::kI32: + return INTEGER(); + case ::substrait::Type::KindCase::kI64: + return BIGINT(); + case ::substrait::Type::KindCase::kFp32: + return REAL(); + case ::substrait::Type::KindCase::kFp64: + return DOUBLE(); + case ::substrait::Type::KindCase::kString: + return VARCHAR(); + case ::substrait::Type::KindCase::kBinary: + return VARBINARY(); case ::substrait::Type::KindCase::kStruct: { - // The type name of struct is in the format of: - // ROW,...typen:namen>. - typeName = "ROW<"; const auto& substraitStruct = substraitType.struct_(); const auto& structTypes = substraitStruct.types(); const auto& structNames = substraitStruct.names(); bool nameProvided = structTypes.size() == structNames.size(); + std::vector types; + std::vector names; for (int i = 0; i < structTypes.size(); i++) { - if (i > 0) { - typeName += ','; - } - typeName += parseType(structTypes[i]).type; - // Struct names could be empty. - if (nameProvided) { - typeName += (':' + structNames[i]); + types.emplace_back(parseType(structTypes[i])); + std::string fieldName = nameProvided ? structNames[i] : "col_" + std::to_string(i); + if (asLowerCase) { + folly::toLowerAscii(fieldName); } + names.emplace_back(fieldName); } - typeName += '>'; - nullability = substraitType.struct_().nullability(); - break; + return ROW(std::move(names), std::move(types)); } case ::substrait::Type::KindCase::kList: { - // The type name of list is in the format of: ARRAY. - const auto& sList = substraitType.list(); - const auto& sType = sList.type(); - typeName = "ARRAY<" + parseType(sType).type + ">"; - nullability = substraitType.list().nullability(); - break; + const auto& fieldType = substraitType.list().type(); + return ARRAY(parseType(fieldType)); } case ::substrait::Type::KindCase::kMap: { - // The type name of map is in the format of: MAP. const auto& sMap = substraitType.map(); const auto& keyType = sMap.key(); const auto& valueType = sMap.value(); - typeName = "MAP<" + parseType(keyType).type + "," + parseType(valueType).type + ">"; - nullability = substraitType.map().nullability(); - break; + return MAP(parseType(keyType), parseType(valueType)); } - case ::substrait::Type::KindCase::kUserDefined: { + case ::substrait::Type::KindCase::kUserDefined: // We only support UNKNOWN type to handle the null literal whose type is // not known. - VELOX_CHECK_EQ(substraitType.user_defined().type_reference(), 0); - typeName = "UNKNOWN"; - nullability = substraitType.string().nullability(); - break; - } - case ::substrait::Type::KindCase::kDate: { - typeName = "DATE"; - nullability = substraitType.date().nullability(); - break; - } - case ::substrait::Type::KindCase::kTimestamp: { - typeName = "TIMESTAMP"; - nullability = substraitType.timestamp().nullability(); - break; - } + return UNKNOWN(); + case ::substrait::Type::KindCase::kDate: + return DATE(); + case ::substrait::Type::KindCase::kTimestamp: + return TIMESTAMP(); case ::substrait::Type::KindCase::kDecimal: { auto precision = substraitType.decimal().precision(); auto scale = substraitType.decimal().scale(); - if (precision <= 18) { - typeName = "SHORT_DECIMAL<" + std::to_string(precision) + "," + std::to_string(scale) + ">"; - } else { - typeName = "LONG_DECIMAL<" + std::to_string(precision) + "," + std::to_string(scale) + ">"; - } - - nullability = substraitType.decimal().nullability(); - break; + return DECIMAL(precision, scale); } default: VELOX_NYI("Parsing for Substrait type not supported: {}", substraitType.DebugString()); } - - bool nullable; - switch (nullability) { - case ::substrait::Type_Nullability::Type_Nullability_NULLABILITY_UNSPECIFIED: - nullable = true; - break; - case ::substrait::Type_Nullability::Type_Nullability_NULLABILITY_NULLABLE: - nullable = true; - break; - case ::substrait::Type_Nullability::Type_Nullability_NULLABILITY_REQUIRED: - nullable = false; - break; - default: - VELOX_NYI("Substrait parsing for nullability {} not supported.", nullability); - } - return SubstraitType{typeName, nullable}; } -std::string SubstraitParser::parseType(const std::string& substraitType) { - auto it = typeMap_.find(substraitType); - if (it == typeMap_.end()) { - VELOX_NYI("Substrait parsing for type {} not supported.", substraitType); - } - return it->second; -}; - -std::vector> SubstraitParser::parseNamedStruct( - const ::substrait::NamedStruct& namedStruct) { +std::vector SubstraitParser::parseNamedStruct(const ::substrait::NamedStruct& namedStruct, bool asLowerCase) { // Note that "names" are not used. // Parse Struct. const auto& substraitStruct = namedStruct.struct_(); const auto& substraitTypes = substraitStruct.types(); - std::vector> substraitTypeList; - substraitTypeList.reserve(substraitTypes.size()); + std::vector typeList; + typeList.reserve(substraitTypes.size()); for (const auto& type : substraitTypes) { - substraitTypeList.emplace_back(std::make_shared(parseType(type))); + typeList.emplace_back(parseType(type, asLowerCase)); } - return substraitTypeList; + return typeList; } std::vector SubstraitParser::parsePartitionColumns(const ::substrait::NamedStruct& namedStruct) { @@ -263,70 +181,73 @@ std::string SubstraitParser::findFunctionSpec( return x->second; } -std::string SubstraitParser::getSubFunctionName(const std::string& subFuncSpec) { - // Get the position of ":" in the function name. - std::size_t pos = subFuncSpec.find(":"); +// TODO Refactor using Bison. +std::string SubstraitParser::getNameBeforeDelimiter(const std::string& signature, const std::string& delimiter) { + std::size_t pos = signature.find(delimiter); if (pos == std::string::npos) { - return subFuncSpec; + return signature; } - return subFuncSpec.substr(0, pos); + return signature.substr(0, pos); } -void SubstraitParser::getSubFunctionTypes(const std::string& subFuncSpec, std::vector& types) { +std::vector SubstraitParser::getSubFunctionTypes(const std::string& substraitFunction) { // Get the position of ":" in the function name. - std::size_t pos = subFuncSpec.find(":"); + size_t pos = substraitFunction.find(":"); // Get the parameter types. - std::string funcTypes; - if (pos == std::string::npos) { - funcTypes = subFuncSpec; - } else { - if (pos == subFuncSpec.size() - 1) { - return; - } - funcTypes = subFuncSpec.substr(pos + 1); + std::vector types; + if (pos == std::string::npos || pos == substraitFunction.size() - 1) { + return types; } - // Split the types with delimiter. - std::string delimiter = "_"; - while ((pos = funcTypes.find(delimiter)) != std::string::npos) { - auto type = funcTypes.substr(0, pos); - if (type != "opt" && type != "req") { - types.emplace_back(type); + // Extract input types with delimiter. + for (;;) { + const size_t endPos = substraitFunction.find("_", pos + 1); + if (endPos == std::string::npos) { + std::string typeName = substraitFunction.substr(pos + 1); + if (typeName != "opt" && typeName != "req") { + types.emplace_back(typeName); + } + break; + } + + const std::string typeName = substraitFunction.substr(pos + 1, endPos - pos - 1); + if (typeName != "opt" && typeName != "req") { + types.emplace_back(typeName); } - funcTypes.erase(0, pos + delimiter.length()); + pos = endPos; } - types.emplace_back(funcTypes); + return types; } std::string SubstraitParser::findVeloxFunction( const std::unordered_map& functionMap, uint64_t id) { std::string funcSpec = findFunctionSpec(functionMap, id); - std::string_view funcName = getNameBeforeDelimiter(funcSpec, ":"); - std::vector types; - getSubFunctionTypes(funcSpec, types); + std::string funcName = getNameBeforeDelimiter(funcSpec); + std::vector types = getSubFunctionTypes(funcSpec); bool isDecimal = false; - for (auto& type : types) { + for (const auto& type : types) { if (type.find("dec") != std::string::npos) { isDecimal = true; break; } } - return mapToVeloxFunction({funcName.begin(), funcName.end()}, isDecimal); + return mapToVeloxFunction(funcName, isDecimal); } std::string SubstraitParser::mapToVeloxFunction(const std::string& substraitFunction, bool isDecimal) { auto it = substraitVeloxFunctionMap_.find(substraitFunction); if (isDecimal) { - if (substraitFunction == "add" || substraitFunction == "subtract" || substraitFunction == "multiply" || - substraitFunction == "divide" || substraitFunction == "avg" || substraitFunction == "avg_merge" || - substraitFunction == "sum" || substraitFunction == "sum_merge" || substraitFunction == "round") { - return "decimal_" + substraitFunction; + if (substraitFunction == "lt" || substraitFunction == "lte" || substraitFunction == "gt" || + substraitFunction == "gte" || substraitFunction == "equal") { + return "decimal_" + it->second; + } + if (substraitFunction == "round") { + return "decimal_round"; } } if (it != substraitVeloxFunctionMap_.end()) { return it->second; } - // If not finding the mapping from Substrait function name to Velox function // name, the original Substrait function name will be used. return substraitFunction; @@ -346,6 +267,16 @@ bool SubstraitParser::configSetInOptimization( return false; } +std::vector SubstraitParser::sigToTypes(const std::string& signature) { + std::vector typeStrs = SubstraitParser::getSubFunctionTypes(signature); + std::vector types; + types.reserve(typeStrs.size()); + for (const auto& typeStr : typeStrs) { + types.emplace_back(VeloxSubstraitSignature::fromSubstraitSignature(typeStr)); + } + return types; +} + std::unordered_map SubstraitParser::substraitVeloxFunctionMap_ = { {"is_not_null", "isnotnull"}, /*Spark functions.*/ {"is_null", "isnull"}, @@ -360,13 +291,14 @@ std::unordered_map SubstraitParser::substraitVeloxFunc {"strpos", "instr"}, {"ends_with", "endswith"}, {"starts_with", "startswith"}, - {"datediff", "date_diff"}, {"named_struct", "row_constructor"}, {"bit_or", "bitwise_or_agg"}, {"bit_or_merge", "bitwise_or_agg_merge"}, {"bit_and", "bitwise_and_agg"}, {"bit_and_merge", "bitwise_and_agg_merge"}, {"collect_set", "array_distinct"}, + {"murmur3hash", "hash_with_seed"}, + {"make_decimal", "make_decimal_by_unscaled_value"}, {"modulus", "mod"} /*Presto functions.*/ }; @@ -383,6 +315,6 @@ const std::unordered_map SubstraitParser::typeMap_ = { {"str", "VARCHAR"}, {"vbin", "VARBINARY"}, {"decShort", "SHORT_DECIMAL"}, - {"decLong", "LONG_DECIMAL"}}; + {"decLong", "HUGEINT"}}; } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitParser.h b/cpp/velox/substrait/SubstraitParser.h index 8bb338ba7bff..fb39d65933af 100644 --- a/cpp/velox/substrait/SubstraitParser.h +++ b/cpp/velox/substrait/SubstraitParser.h @@ -28,29 +28,24 @@ #include +#include "velox/type/Type.h" + namespace gluten { /// This class contains some common functions used to parse Substrait /// components, and convert them into recognizable representations. class SubstraitParser { public: - /// Stores the type name and nullability. - struct SubstraitType { - std::string type; - bool nullable; - }; - /// Used to parse Substrait NamedStruct. - static std::vector> parseNamedStruct(const ::substrait::NamedStruct& namedStruct); + static std::vector parseNamedStruct( + const ::substrait::NamedStruct& namedStruct, + bool asLowerCase = false); /// Used to parse partition columns from Substrait NamedStruct. static std::vector parsePartitionColumns(const ::substrait::NamedStruct& namedStruct); - /// Parse Substrait Type. - static SubstraitType parseType(const ::substrait::Type& substraitType); - - // Parse substraitType type such as i32. - static std::string parseType(const std::string& substraitType); + /// Parse Substrait Type to Velox type. + static facebook::velox::TypePtr parseType(const ::substrait::Type& substraitType, bool asLowerCase = false); /// Parse Substrait ReferenceSegment. static int32_t parseReferenceSegment(const ::substrait::Expression::ReferenceSegment& refSegment); @@ -74,12 +69,11 @@ class SubstraitParser { /// specifications in Substrait yaml files. static std::string findFunctionSpec(const std::unordered_map& functionMap, uint64_t id); - /// Extracts the function name for a function from specified compound name. - /// When the input is a simple name, it will be returned. - static std::string getSubFunctionName(const std::string& functionSpec); + /// Extracts the name of a function by splitting signature with delimiter. + static std::string getNameBeforeDelimiter(const std::string& signature, const std::string& delimiter = ":"); /// This function is used get the types from the compound name. - static void getSubFunctionTypes(const std::string& subFuncSpec, std::vector& types); + static std::vector getSubFunctionTypes(const std::string& subFuncSpec); /// Used to find the Velox function name according to the function id /// from a pre-constructed function map. @@ -95,6 +89,9 @@ class SubstraitParser { /// @return Whether the config is set as true. static bool configSetInOptimization(const ::substrait::extensions::AdvancedExtension&, const std::string& config); + /// Extract input types from Substrait function signature. + static std::vector sigToTypes(const std::string& functionSig); + private: /// A map used for mapping Substrait function keywords into Velox functions' /// keywords. Key: the Substrait function keyword, Value: the Velox function diff --git a/cpp/velox/substrait/SubstraitToVeloxExpr.cc b/cpp/velox/substrait/SubstraitToVeloxExpr.cc index 5c4618c3bdb5..e2ad05c81032 100644 --- a/cpp/velox/substrait/SubstraitToVeloxExpr.cc +++ b/cpp/velox/substrait/SubstraitToVeloxExpr.cc @@ -72,11 +72,6 @@ Timestamp getLiteralValue(const ::substrait::Expression::Literal& literal) { return Timestamp::fromMicros(literal.timestamp()); } -template <> -Date getLiteralValue(const ::substrait::Expression::Literal& literal) { - return Date(literal.date()); -} - ArrayVectorPtr makeArrayVector(const VectorPtr& elements) { BufferPtr offsets = allocateOffsets(1, elements->pool()); BufferPtr sizes = allocateOffsets(1, elements->pool()); @@ -142,6 +137,9 @@ void setLiteralValue(const ::substrait::Expression::Literal& literal, FlatVector } else { VELOX_FAIL("Unexpected string or binary literal"); } + } else if (vector->type()->isDate()) { + auto dateVector = vector->template asFlatVector(); + dateVector->set(index, int(literal.date())); } else { vector->set(index, getLiteralValue(literal)); } @@ -280,7 +278,7 @@ core::TypedExprPtr SubstraitVeloxExprConverter::toLambdaExpr( SubstraitParser::findVeloxFunction(functionMap_, arg.scalar_function().function_reference()); CHECK_EQ(veloxFunction, "namedlambdavariable"); argumentNames.emplace_back(arg.scalar_function().arguments(0).value().literal().string()); - argumentTypes.emplace_back(substraitTypeToVeloxType(substraitFunc.output_type())); + argumentTypes.emplace_back(SubstraitParser::parseType(substraitFunc.output_type())); } auto rowType = ROW(std::move(argumentNames), std::move(argumentTypes)); // Arg[0] -> function. @@ -298,16 +296,16 @@ core::TypedExprPtr SubstraitVeloxExprConverter::toVeloxExpr( params.emplace_back(toVeloxExpr(sArg.value(), inputType)); } const auto& veloxFunction = SubstraitParser::findVeloxFunction(functionMap_, substraitFunc.function_reference()); - std::string typeName = SubstraitParser::parseType(substraitFunc.output_type()).type; + const auto& outputType = SubstraitParser::parseType(substraitFunc.output_type()); if (veloxFunction == "lambdafunction") { return toLambdaExpr(substraitFunc, inputType); } else if (veloxFunction == "namedlambdavariable") { - return makeFieldAccessExpr(substraitFunc.arguments(0).value().literal().string(), toVeloxType(typeName), nullptr); + return makeFieldAccessExpr(substraitFunc.arguments(0).value().literal().string(), outputType, nullptr); } else if (veloxFunction == "extract") { - return toExtractExpr(std::move(params), toVeloxType(typeName)); + return toExtractExpr(std::move(params), outputType); } else { - return std::make_shared(toVeloxType(typeName), std::move(params), veloxFunction); + return std::make_shared(outputType, std::move(params), veloxFunction); } } @@ -318,15 +316,15 @@ std::shared_ptr SubstraitVeloxExprConverter::lite VELOX_CHECK_GE(literals.size(), 0, "List should have at least one item."); std::optional literalType = std::nullopt; for (const auto& literal : literals) { - auto veloxVariant = toVeloxExpr(literal)->value(); + auto veloxVariant = toVeloxExpr(literal); if (!literalType.has_value()) { - literalType = veloxVariant.inferType(); + literalType = veloxVariant->type(); } - variants.emplace_back(veloxVariant); + variants.emplace_back(veloxVariant->value()); } VELOX_CHECK(literalType.has_value(), "Type expected."); auto varArray = variant::array(variants); - ArrayVectorPtr arrayVector = variantArrayToVector(varArray.inferType(), varArray.array(), pool_); + ArrayVectorPtr arrayVector = variantArrayToVector(ARRAY(literalType.value()), varArray.array(), pool_); // Wrap the array vector into constant vector. auto constantVector = BaseVector::wrapInConstant(1 /*length*/, 0 /*index*/, arrayVector); return std::make_shared(constantVector); @@ -375,7 +373,7 @@ std::shared_ptr SubstraitVeloxExprConverter::toVe case ::substrait::Expression_Literal::LiteralTypeCase::kString: return std::make_shared(VARCHAR(), variant(substraitLit.string())); case ::substrait::Expression_Literal::LiteralTypeCase::kDate: - return std::make_shared(DATE(), variant(Date(substraitLit.date()))); + return std::make_shared(DATE(), variant(int(substraitLit.date()))); case ::substrait::Expression_Literal::LiteralTypeCase::kTimestamp: return std::make_shared( TIMESTAMP(), variant(Timestamp::fromMicros(substraitLit.timestamp()))); @@ -412,7 +410,7 @@ std::shared_ptr SubstraitVeloxExprConverter::toVe } } case ::substrait::Expression_Literal::LiteralTypeCase::kNull: { - auto veloxType = substraitTypeToVeloxType(substraitLit.null()); + auto veloxType = SubstraitParser::parseType(substraitLit.null()); if (veloxType->isShortDecimal()) { return std::make_shared(veloxType, variant::null(TypeKind::BIGINT)); } else if (veloxType->isLongDecimal()) { @@ -475,12 +473,12 @@ VectorPtr SubstraitVeloxExprConverter::literalsToVector( case ::substrait::Expression_Literal::LiteralTypeCase::kVarChar: return constructFlatVector(elementAtFunc, childSize, VARCHAR(), pool_); case ::substrait::Expression_Literal::LiteralTypeCase::kNull: { - auto veloxType = substraitTypeToVeloxType(literal.null()); + auto veloxType = SubstraitParser::parseType(literal.null()); auto kind = veloxType->kind(); return VELOX_DYNAMIC_SCALAR_TYPE_DISPATCH(constructFlatVector, kind, elementAtFunc, childSize, veloxType, pool_); } case ::substrait::Expression_Literal::LiteralTypeCase::kDate: - return constructFlatVector(elementAtFunc, childSize, DATE(), pool_); + return constructFlatVector(elementAtFunc, childSize, DATE(), pool_); case ::substrait::Expression_Literal::LiteralTypeCase::kTimestamp: return constructFlatVector(elementAtFunc, childSize, TIMESTAMP(), pool_); case ::substrait::Expression_Literal::LiteralTypeCase::kIntervalDayToSecond: @@ -539,7 +537,7 @@ RowVectorPtr SubstraitVeloxExprConverter::literalsToRowVector(const ::substrait: core::TypedExprPtr SubstraitVeloxExprConverter::toVeloxExpr( const ::substrait::Expression::Cast& castExpr, const RowTypePtr& inputType) { - auto type = substraitTypeToVeloxType(castExpr.type()); + auto type = SubstraitParser::parseType(castExpr.type()); bool nullOnFailure = isNullOnFailure(castExpr.failure_behavior()); std::vector inputs{toVeloxExpr(castExpr.input(), inputType)}; @@ -609,11 +607,11 @@ std::unordered_map SubstraitVeloxExprConverter::extrac {"MINUTE", "minute"}, {"HOUR", "hour"}, {"DAY", "day"}, - {"DAY_OF_WEEK", "day_of_week"}, - {"DAY_OF_YEAR", "day_of_year"}, + {"DAY_OF_WEEK", "dayofweek"}, + {"DAY_OF_YEAR", "dayofyear"}, {"MONTH", "month"}, {"QUARTER", "quarter"}, {"YEAR", "year"}, - {"YEAR_OF_WEEK", "year_of_week"}}; + {"YEAR_OF_WEEK", "week_of_year"}}; } // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 59afd9db844c..f38c2eefacca 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -355,9 +355,12 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: for (const auto& arg : aggFunction.arguments()) { aggParams.emplace_back(exprConverter_->toVeloxExpr(arg.value(), inputType)); } - auto aggVeloxType = substraitTypeToVeloxType(aggFunction.output_type()); + auto aggVeloxType = SubstraitParser::parseType(aggFunction.output_type()); auto aggExpr = std::make_shared(aggVeloxType, std::move(aggParams), funcName); - aggregates.emplace_back(core::AggregationNode::Aggregate{aggExpr, mask, {}, {}}); + + std::vector rawInputTypes = + SubstraitParser::sigToTypes(SubstraitParser::findFunctionSpec(functionMap_, aggFunction.function_reference())); + aggregates.emplace_back(core::AggregationNode::Aggregate{aggExpr, rawInputTypes, mask, {}, {}}); } bool ignoreNullKeys = false; @@ -608,7 +611,7 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: for (const auto& arg : windowFunction.arguments()) { windowParams.emplace_back(exprConverter_->toVeloxExpr(arg.value(), inputType)); } - auto windowVeloxType = substraitTypeToVeloxType(windowFunction.output_type()); + auto windowVeloxType = SubstraitParser::parseType(windowFunction.output_type()); auto windowCall = std::make_shared(windowVeloxType, std::move(windowParams), funcName); auto upperBound = windowFunction.upper_bound(); auto lowerBound = windowFunction.lower_bound(); @@ -633,8 +636,29 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } auto [sortingKeys, sortingOrders] = processSortField(windowRel.sorts(), inputType); - return std::make_shared( - nextPlanNodeId(), partitionKeys, sortingKeys, sortingOrders, windowColumnNames, windowNodeFunctions, childNode); + + if (windowRel.has_advanced_extension() && + SubstraitParser::configSetInOptimization(windowRel.advanced_extension(), "isStreaming=")) { + return std::make_shared( + nextPlanNodeId(), + partitionKeys, + sortingKeys, + sortingOrders, + windowColumnNames, + windowNodeFunctions, + true /*inputsSorted*/, + childNode); + } else { + return std::make_shared( + nextPlanNodeId(), + partitionKeys, + sortingKeys, + sortingOrders, + windowColumnNames, + windowNodeFunctions, + false /*inputsSorted*/, + childNode); + } } core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait::SortRel& sortRel) { @@ -769,12 +793,8 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: } colNameList.emplace_back(fieldName); } - auto substraitTypeList = SubstraitParser::parseNamedStruct(baseSchema); + veloxTypeList = SubstraitParser::parseNamedStruct(baseSchema, asLowerCase); isPartitionColumns = SubstraitParser::parsePartitionColumns(baseSchema); - veloxTypeList.reserve(substraitTypeList.size()); - for (const auto& substraitType : substraitTypeList) { - veloxTypeList.emplace_back(toVeloxType(substraitType->type, asLowerCase)); - } } // Parse local files and construct split info. @@ -1038,7 +1058,7 @@ void SubstraitToVeloxPlanConverter::flattenConditions( const auto& sFunc = substraitFilter.scalar_function(); auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, sFunc.function_reference()); // TODO: Only and relation is supported here. - if (SubstraitParser::getSubFunctionName(filterNameSpec) == "and") { + if (SubstraitParser::getNameBeforeDelimiter(filterNameSpec) == "and") { for (const auto& sCondition : sFunc.arguments()) { flattenConditions(sCondition.value(), scalarFunctions, singularOrLists, ifThens); } @@ -1104,13 +1124,13 @@ void SubstraitToVeloxPlanConverter::extractJoinKeys( auto visited = expressions.back(); expressions.pop_back(); if (visited->rex_type_case() == ::substrait::Expression::RexTypeCase::kScalarFunction) { - const auto& funcName = SubstraitParser::getSubFunctionName( + const auto& funcName = SubstraitParser::getNameBeforeDelimiter( SubstraitParser::findVeloxFunction(functionMap_, visited->scalar_function().function_reference())); const auto& args = visited->scalar_function().arguments(); if (funcName == "and") { expressions.push_back(&args[0].value()); expressions.push_back(&args[1].value()); - } else if (funcName == "eq" || funcName == "equalto") { + } else if (funcName == "eq" || funcName == "equalto" || funcName == "decimal_equalto") { VELOX_CHECK(std::all_of(args.cbegin(), args.cend(), [](const ::substrait::FunctionArgument& arg) { return arg.value().has_selection(); })); @@ -1136,7 +1156,7 @@ connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::createSubfieldFi // Process scalarFunctions. for (const auto& scalarFunction : scalarFunctions) { auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getSubFunctionName(filterNameSpec); + auto filterName = SubstraitParser::getNameBeforeDelimiter(filterNameSpec); if (filterName == sNot) { VELOX_CHECK(scalarFunction.arguments().size() == 1); @@ -1279,7 +1299,7 @@ bool SubstraitToVeloxPlanConverter::canPushdownNot( auto argFunction = SubstraitParser::findFunctionSpec(functionMap_, notArg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getSubFunctionName(argFunction); + auto functionName = SubstraitParser::getNameBeforeDelimiter(argFunction); static const std::unordered_set supportedNotFunctions = {sGte, sGt, sLte, sLt, sEqual}; @@ -1308,7 +1328,7 @@ bool SubstraitToVeloxPlanConverter::canPushdownOr( if (arg.value().has_scalar_function()) { auto nameSpec = SubstraitParser::findFunctionSpec(functionMap_, arg.value().scalar_function().function_reference()); - auto functionName = SubstraitParser::getSubFunctionName(nameSpec); + auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); uint32_t fieldIdx; bool isFieldOrWithLiteral = fieldOrWithLiteral(arg.value().scalar_function().arguments(), fieldIdx); @@ -1362,7 +1382,7 @@ void SubstraitToVeloxPlanConverter::separateFilters( for (const auto& scalarFunction : scalarFunctions) { auto filterNameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto filterName = SubstraitParser::getSubFunctionName(filterNameSpec); + 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(); @@ -1497,7 +1517,7 @@ void SubstraitToVeloxPlanConverter::setFilterInfo( std::vector& columnToFilterInfo, bool reverse) { auto nameSpec = SubstraitParser::findFunctionSpec(functionMap_, scalarFunction.function_reference()); - auto functionName = SubstraitParser::getSubFunctionName(nameSpec); + auto functionName = SubstraitParser::getNameBeforeDelimiter(nameSpec); // Extract the column index and column bound from the scalar function. std::optional colIdx; @@ -1541,6 +1561,13 @@ void SubstraitToVeloxPlanConverter::setFilterInfo( std::optional val; auto inputType = inputTypeList[colIdxVal]; + if (inputType->isDate()) { + if (substraitLit) { + val = variant(int(substraitLit.value().date())); + } + setColumnFilterInfo(functionName, val, columnToFilterInfo[colIdxVal], reverse); + return; + } switch (inputType->kind()) { case TypeKind::TINYINT: if (substraitLit) { @@ -1589,11 +1616,6 @@ void SubstraitToVeloxPlanConverter::setFilterInfo( val = variant(substraitLit.value().string()); } break; - case TypeKind::DATE: - if (substraitLit) { - val = variant(Date(substraitLit.value().date())); - } - break; case TypeKind::HUGEINT: if (substraitLit) { if (inputType->isLongDecimal()) { @@ -1607,9 +1629,8 @@ void SubstraitToVeloxPlanConverter::setFilterInfo( } break; case TypeKind::ARRAY: - // Doing nothing here can let filter IsNotNull still work. - break; case TypeKind::MAP: + case TypeKind::ROW: // Doing nothing here can let filter IsNotNull still work. break; default: @@ -1627,24 +1648,36 @@ void SubstraitToVeloxPlanConverter::createNotEqualFilter( using NativeType = typename RangeTraits::NativeType; using RangeType = typename RangeTraits::RangeType; // Value > lower - std::unique_ptr lowerFilter = std::make_unique( - notVariant.value(), /*lower*/ - false, /*lowerUnbounded*/ - true, /*lowerExclusive*/ - getMax(), /*upper*/ - true, /*upperUnbounded*/ - false, /*upperExclusive*/ - nullAllowed); /*nullAllowed*/ + std::unique_ptr lowerFilter; + if constexpr (std::is_same_v) { + 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 = std::make_unique( - getLowest(), /*lower*/ - true, /*lowerUnbounded*/ - false, /*lowerExclusive*/ - notVariant.value(), /*upper*/ - false, /*upperUnbounded*/ - true, /*upperExclusive*/ - nullAllowed); /*nullAllowed*/ + std::unique_ptr upperFilter; + if constexpr (std::is_same_v) { + 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. @@ -1671,7 +1704,7 @@ void SubstraitToVeloxPlanConverter::setInFilter( int64_t value = variant.value(); values.emplace_back(value); } - filters[common::Subfield(inputName, true)] = common::createBigintValues(values, nullAllowed); + filters[common::Subfield(inputName)] = common::createBigintValues(values, nullAllowed); } template <> @@ -1688,7 +1721,7 @@ void SubstraitToVeloxPlanConverter::setInFilter( int64_t value = variant.value(); values.emplace_back(value); } - filters[common::Subfield(inputName, true)] = common::createBigintValues(values, nullAllowed); + filters[common::Subfield(inputName)] = common::createBigintValues(values, nullAllowed); } template <> @@ -1705,7 +1738,7 @@ void SubstraitToVeloxPlanConverter::setInFilter( int64_t value = variant.value(); values.emplace_back(value); } - filters[common::Subfield(inputName, true)] = common::createBigintValues(values, nullAllowed); + filters[common::Subfield(inputName)] = common::createBigintValues(values, nullAllowed); } template <> @@ -1722,24 +1755,7 @@ void SubstraitToVeloxPlanConverter::setInFilter( int64_t value = variant.value(); values.emplace_back(value); } - filters[common::Subfield(inputName, true)] = common::createBigintValues(values, nullAllowed); -} - -template <> -void SubstraitToVeloxPlanConverter::setInFilter( - const std::vector& variants, - bool nullAllowed, - 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 int32 to get value from date variant. - int64_t value = variant.value(); - values.emplace_back(value); - } - filters[common::Subfield(inputName, true)] = common::createBigintValues(values, nullAllowed); + filters[common::Subfield(inputName)] = common::createBigintValues(values, nullAllowed); } template <> @@ -1754,7 +1770,7 @@ void SubstraitToVeloxPlanConverter::setInFilter( std::string value = variant.value(); values.emplace_back(value); } - filters[common::Subfield(inputName, true)] = std::make_unique(values, nullAllowed); + filters[common::Subfield(inputName)] = std::make_unique(values, nullAllowed); } template @@ -1766,7 +1782,7 @@ void SubstraitToVeloxPlanConverter::setSubfieldFilter( using MultiRangeType = typename RangeTraits::MultiRangeType; if (colFilters.size() == 1) { - filters[common::Subfield(inputName, true)] = std::move(colFilters[0]); + filters[common::Subfield(inputName)] = std::move(colFilters[0]); } else if (colFilters.size() > 1) { // BigintMultiRange should have been sorted if (colFilters[0]->kind() == common::FilterKind::kBigintRange) { @@ -1775,8 +1791,12 @@ void SubstraitToVeloxPlanConverter::setSubfieldFilter( dynamic_cast(b.get())->lower(); }); } - - filters[common::Subfield(inputName, true)] = std::make_unique(std::move(colFilters), nullAllowed); + if constexpr (std::is_same_v) { + filters[common::Subfield(inputName)] = + std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); + } else { + filters[common::Subfield(inputName)] = std::make_unique(std::move(colFilters), nullAllowed); + } } } @@ -1800,7 +1820,7 @@ void SubstraitToVeloxPlanConverter::constructSubfieldFilters( } else if constexpr (KIND == facebook::velox::TypeKind::ARRAY || KIND == facebook::velox::TypeKind::MAP) { // Only IsNotNull filter is supported for the above two type kinds now. if (rangeSize == 0 && !nullAllowed) { - filters[common::Subfield(inputName, true)] = std::move(std::make_unique()); + filters[common::Subfield(inputName)] = std::move(std::make_unique()); } else { VELOX_NYI("constructSubfieldFilters only support IsNotNull for input type '{}'", inputType); } @@ -1831,14 +1851,19 @@ void SubstraitToVeloxPlanConverter::constructSubfieldFilters( // 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."); - filters[common::Subfield(inputName, true)] = std::make_unique(std::move(colFilters), nullAllowed); + if constexpr (std::is_same_v) { + filters[common::Subfield(inputName)] = + std::make_unique(std::move(colFilters), nullAllowed, true /*nanAllowed*/); + } else { + filters[common::Subfield(inputName)] = std::make_unique(std::move(colFilters), nullAllowed); + } return; } // Handle null filtering. if (rangeSize == 0 && !nullAllowed) { std::unique_ptr filter = std::make_unique(); - filters[common::Subfield(inputName, true)] = std::move(filter); + filters[common::Subfield(inputName)] = std::move(filter); return; } @@ -1885,8 +1910,14 @@ void SubstraitToVeloxPlanConverter::constructSubfieldFilters( upperExclusive = filterInfo.upperExclusives_[idx]; } - std::unique_ptr filter = std::move(std::make_unique( - lowerBound, lowerUnbounded, lowerExclusive, upperBound, upperUnbounded, upperExclusive, nullAllowed)); + 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)); } @@ -1917,58 +1948,61 @@ connector::hive::SubfieldFilters SubstraitToVeloxPlanConverter::mapToFilters( // Construct the subfield filters based on the filter info map. connector::hive::SubfieldFilters filters; for (uint32_t colIdx = 0; colIdx < inputNameList.size(); colIdx++) { - auto inputType = inputTypeList[colIdx]; - 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: + if (columnToFilterInfo[colIdx].isInitialized()) { + auto inputType = inputTypeList[colIdx]; + if (inputType->isDate()) { 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::DATE: - 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; - default: - VELOX_NYI("Subfield filters creation not supported for input type '{}' in mapToFilters", inputType); + 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; + default: + VELOX_NYI("Subfield filters creation not supported for input type '{}' in mapToFilters", inputType); + } } } diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.h b/cpp/velox/substrait/SubstraitToVeloxPlan.h index 6b73f1c212ca..f8ad7d072725 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.h @@ -20,7 +20,9 @@ #include "SubstraitToVeloxExpr.h" #include "TypeUtils.h" #include "velox/connectors/hive/HiveConnector.h" +#include "velox/connectors/hive/TableHandle.h" #include "velox/core/PlanNode.h" +#include "velox/dwio/common/Options.h" namespace gluten { @@ -150,6 +152,12 @@ class SubstraitToVeloxPlanConverter { /// Get aggregation step from AggregateRel. core::AggregationNode::Step toAggregationStep(const ::substrait::AggregateRel& sAgg); + /// Helper Function to convert Substrait sortField to Velox sortingKeys and + /// sortingOrders. + std::pair, std::vector> processSortField( + const ::google::protobuf::RepeatedPtrField<::substrait::SortField>& sortField, + const RowTypePtr& inputType); + private: /// Integrate Substrait emit feature. Here a given 'substrait::RelCommon' /// is passed and check if emit is defined for this relation. Basically a @@ -326,12 +334,6 @@ class SubstraitToVeloxPlanConverter { std::vector values_; }; - /// Helper Function to convert Substrait sortField to Velox sortingKeys and - /// sortingOrders. - std::pair, std::vector> processSortField( - const ::google::protobuf::RepeatedPtrField<::substrait::SortField>& sortField, - const RowTypePtr& inputType); - /// Returns unique ID to use for plan node. Produces sequential numbers /// starting from zero. std::string nextPlanNodeId(); diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc index 7a1d3565f7d5..e5a45951a6bd 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.cc @@ -21,7 +21,9 @@ #include #include "TypeUtils.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" #include "velox/type/Tokenizer.h" @@ -39,27 +41,31 @@ static const std::unordered_set kBlackList = { "split_part", "factorial", "concat_ws", + "from_json", "rand", "json_array_length", "from_unixtime", "repeat", - "translate", - "add_months", "date_format", "trunc", "sequence", "posexplode", "arrays_overlap", - "array_min", - "array_max", "approx_percentile"}; bool validateColNames(const ::substrait::NamedStruct& schema) { - for (auto& name : schema.names()) { - common::Tokenizer token(name); + // Keeps the same logic with 'Tokenizer::isUnquotedPathCharacter' at + // https://github.com/oap-project/velox/blob/update/velox/type/Tokenizer.cpp#L154-L157. + auto isUnquotedPathCharacter = [](char c) { + return c == ':' || c == '$' || c == '-' || c == '/' || c == '@' || c == '|' || c == '#' || c == '.' || c == '-' || + c == '_' || isalnum(c); + }; + + for (const auto& name : schema.names()) { + common::Tokenizer token(name, common::Separators::get()); for (auto i = 0; i < name.size(); i++) { auto c = name[i]; - if (!token.isUnquotedPathCharacter(c)) { + if (!isUnquotedPathCharacter(c)) { std::cout << "native validation failed due to: Illegal column charactor " << c << "in column " << name << std::endl; return false; @@ -94,7 +100,7 @@ bool SubstraitToVeloxPlanValidator::validateInputTypes( const auto& sTypes = inputType.struct_().types(); for (const auto& sType : sTypes) { try { - types.emplace_back(substraitTypeToVeloxType(sType)); + types.emplace_back(SubstraitParser::parseType(sType)); } catch (const VeloxException& err) { logValidateMsg("native validation failed due to: Type is not supported, " + err.message()); return false; @@ -195,9 +201,8 @@ bool SubstraitToVeloxPlanValidator::validateScalarFunction( const auto& function = SubstraitParser::findFunctionSpec(planConverter_.getFunctionMap(), scalarFunction.function_reference()); - const auto& name = SubstraitParser::getSubFunctionName(function); - std::vector types; - SubstraitParser::getSubFunctionTypes(function, types); + const auto& name = SubstraitParser::getNameBeforeDelimiter(function); + std::vector types = SubstraitParser::getSubFunctionTypes(function); if (name == "round") { return validateRound(scalarFunction, inputType); @@ -284,7 +289,7 @@ bool SubstraitToVeloxPlanValidator::validateCast( return false; } - const auto& toType = substraitTypeToVeloxType(castExpr.type()); + const auto& toType = SubstraitParser::parseType(castExpr.type()); if (toType->kind() == TypeKind::TIMESTAMP) { logValidateMsg("native validation failed due to: Casting to TIMESTAMP is not supported."); return false; @@ -293,6 +298,17 @@ bool SubstraitToVeloxPlanValidator::validateCast( core::TypedExprPtr input = exprConverter_->toVeloxExpr(castExpr.input(), inputType); // Casting from some types is not supported. See CastExpr::applyCast. + if (input->type()->isDate()) { + if (toType->kind() == TypeKind::TIMESTAMP) { + logValidateMsg("native validation failed due to: Casting from DATE to TIMESTAMP is not supported."); + return false; + } + if (toType->kind() != TypeKind::VARCHAR) { + logValidateMsg(fmt::format( + "native validation failed due to: Casting from DATE to {} is not supported.", toType->toString())); + return false; + } + } switch (input->type()->kind()) { case TypeKind::ARRAY: case TypeKind::MAP: @@ -300,18 +316,6 @@ bool SubstraitToVeloxPlanValidator::validateCast( case TypeKind::VARBINARY: logValidateMsg("native validation failed due to: Invalid input type in casting: ARRAY/MAP/ROW/VARBINARY"); return false; - case TypeKind::DATE: { - if (toType->kind() == TypeKind::TIMESTAMP) { - logValidateMsg("native validation failed due to: Casting from DATE to TIMESTAMP is not supported."); - return false; - } - if (toType->kind() != TypeKind::VARCHAR) { - logValidateMsg(fmt::format( - "native validation failed due to: Casting from DATE to {} is not supported.", toType->toString())); - return false; - } - break; - } case TypeKind::TIMESTAMP: { logValidateMsg( "native validation failed due to: Casting from TIMESTAMP is not supported or has incorrect result."); @@ -323,6 +327,15 @@ bool SubstraitToVeloxPlanValidator::validateCast( return true; } +bool SubstraitToVeloxPlanValidator::validateIfThen( + const ::substrait::Expression_IfThen& ifThen, + const RowTypePtr& inputType) { + for (const auto& ifThen : ifThen.ifs()) { + return validateExpression(ifThen.if_(), inputType) && validateExpression(ifThen.then(), inputType); + } + return true; +} + bool SubstraitToVeloxPlanValidator::validateExpression( const ::substrait::Expression& expression, const RowTypePtr& inputType) { @@ -334,28 +347,91 @@ bool SubstraitToVeloxPlanValidator::validateExpression( return validateLiteral(expression.literal(), inputType); case ::substrait::Expression::RexTypeCase::kCast: return validateCast(expression.cast(), inputType); + case ::substrait::Expression::RexTypeCase::kIfThen: + return validateIfThen(expression.if_then(), inputType); default: return true; } } bool SubstraitToVeloxPlanValidator::validate(const ::substrait::FetchRel& fetchRel) { - const auto& extension = fetchRel.advanced_extension(); - std::vector types; - if (!validateInputTypes(extension, types)) { - logValidateMsg("native validation failed due to: unsupported input types in FetchRel."); - return false; + RowTypePtr rowType = nullptr; + // Get and validate the input types from extension. + if (fetchRel.has_advanced_extension()) { + const auto& extension = fetchRel.advanced_extension(); + std::vector types; + if (!validateInputTypes(extension, types)) { + logValidateMsg("native validation failed due to: unsupported input types in ExpandRel."); + return false; + } + + int32_t inputPlanNodeId = 0; + std::vector names; + names.reserve(types.size()); + 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) { logValidateMsg("native validation failed due to: Offset and count should be valid in FetchRel."); return false; } + + // Check the input of fetchRel, if it's sortRel, we need to check whether the sorting key is duplicated. + bool topNFlag = false; + if (fetchRel.has_input()) { + topNFlag = fetchRel.input().has_sort(); + if (topNFlag) { + ::substrait::SortRel sortRel = fetchRel.input().sort(); + auto [sortingKeys, sortingOrders] = planConverter_.processSortField(sortRel.sorts(), rowType); + folly::F14FastSet sortingKeyNames; + for (const auto& sortingKey : sortingKeys) { + auto result = sortingKeyNames.insert(sortingKey->name()); + if (!result.second) { + logValidateMsg( + "native validation failed due to: if the input of fetchRel is a SortRel, we will convert it to a TopNNode. In Velox, it is important to ensure unique sorting keys. However, duplicate keys were found in this case."); + return false; + } + } + } + } + return true; } bool SubstraitToVeloxPlanValidator::validate(const ::substrait::GenerateRel& generateRel) { - // TODO(yuan): add check + if (generateRel.has_input() && !validate(generateRel.input())) { + logValidateMsg("native validation failed due to: input validation fails in GenerateRel."); + return false; + } + + // Get and validate the input types from extension. + if (!generateRel.has_advanced_extension()) { + logValidateMsg("native validation failed due to: Input types are expected in GenerateRel."); + return false; + } + const auto& extension = generateRel.advanced_extension(); + std::vector types; + if (!validateInputTypes(extension, types)) { + logValidateMsg("native validation failed due to: Validation failed for input types in GenerateRel."); + return false; + } + + int32_t inputPlanNodeId = 0; + // Create the fake input names to be used in row type. + std::vector names; + names.reserve(types.size()); + for (uint32_t colIdx = 0; colIdx < types.size(); colIdx++) { + names.emplace_back(SubstraitParser::makeNodeName(inputPlanNodeId, colIdx)); + } + auto rowType = std::make_shared(std::move(names), std::move(types)); + + if (generateRel.has_generator() && !validateExpression(generateRel.generator(), rowType)) { + logValidateMsg("native validation failed due to: input validation fails in GenerateRel."); + return false; + } return true; } @@ -482,7 +558,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo try { const auto& windowFunction = smea.measure(); funcSpecs.emplace_back(planConverter_.findFuncSpec(windowFunction.function_reference())); - substraitTypeToVeloxType(windowFunction.output_type()); + SubstraitParser::parseType(windowFunction.output_type()); for (const auto& arg : windowFunction.arguments()) { auto typeCase = arg.value().rex_type_case(); switch (typeCase) { @@ -523,7 +599,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::WindowRel& windo // Validate supported aggregate functions. static const std::unordered_set unsupportedFuncs = {"collect_list", "collect_set"}; for (const auto& funcSpec : funcSpecs) { - auto funcName = SubstraitParser::getSubFunctionName(funcSpec); + auto funcName = SubstraitParser::getNameBeforeDelimiter(funcSpec); if (unsupportedFuncs.find(funcName) != unsupportedFuncs.end()) { logValidateMsg("native validation failed due to: " + funcName + " was not supported in WindowRel."); return false; @@ -834,55 +910,6 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::JoinRel& joinRel return true; } -TypePtr SubstraitToVeloxPlanValidator::getDecimalType(const std::string& decimalType) { - // Decimal info is in the format of dec. - auto precisionStart = decimalType.find_first_of('<'); - auto tokenIndex = decimalType.find_first_of(','); - auto scaleStart = decimalType.find_first_of('>'); - auto precision = stoi(decimalType.substr(precisionStart + 1, (tokenIndex - precisionStart - 1))); - auto scale = stoi(decimalType.substr(tokenIndex + 1, (scaleStart - tokenIndex - 1))); - return DECIMAL(precision, scale); -} - -TypePtr SubstraitToVeloxPlanValidator::getRowType(const std::string& structType) { - // Struct info is in the format of struct. - // TODO: nested struct is not supported. - auto structStart = structType.find_first_of('<'); - auto structEnd = structType.find_last_of('>'); - if (structEnd - structStart > 1) { - logValidateMsg("native validation failed due to: More information is needed to create RowType"); - } - VELOX_CHECK( - structEnd - structStart > 1, "native validation failed due to: More information is needed to create RowType"); - std::string childrenTypes = structType.substr(structStart + 1, structEnd - structStart - 1); - - // Split the types with delimiter. - std::string delimiter = ","; - std::size_t pos; - std::vector types; - std::vector names; - while ((pos = childrenTypes.find(delimiter)) != std::string::npos) { - const auto& typeStr = childrenTypes.substr(0, pos); - std::string decDelimiter = ">"; - if (typeStr.find("dec") != std::string::npos) { - std::size_t endPos = childrenTypes.find(decDelimiter); - VELOX_CHECK(endPos >= pos + 1, "Decimal scale is expected."); - const auto& decimalStr = typeStr + childrenTypes.substr(pos, endPos - pos) + decDelimiter; - types.emplace_back(getDecimalType(decimalStr)); - names.emplace_back(""); - childrenTypes.erase(0, endPos + delimiter.length() + decDelimiter.length()); - continue; - } - - types.emplace_back(substraitTypeToVeloxType(typeStr)); - names.emplace_back(""); - childrenTypes.erase(0, pos + delimiter.length()); - } - types.emplace_back(substraitTypeToVeloxType(childrenTypes)); - names.emplace_back(""); - return std::make_shared(std::move(names), std::move(types)); -} - bool SubstraitToVeloxPlanValidator::validateAggRelFunctionType(const ::substrait::AggregateRel& aggRel) { if (aggRel.measures_size() == 0) { return true; @@ -894,21 +921,11 @@ bool SubstraitToVeloxPlanValidator::validateAggRelFunctionType(const ::substrait std::vector types; bool isDecimal = false; try { - std::vector funcTypes; - SubstraitParser::getSubFunctionTypes(funcSpec, funcTypes); - types.reserve(funcTypes.size()); - for (auto& type : funcTypes) { - if (!isDecimal && type.find("dec") != std::string::npos) { + types = SubstraitParser::sigToTypes(funcSpec); + for (const auto& type : types) { + if (!isDecimal && type->isDecimal()) { isDecimal = true; } - - if (type.find("struct") != std::string::npos) { - types.emplace_back(getRowType(type)); - } else if (type.find("dec") != std::string::npos) { - types.emplace_back(getDecimalType(type)); - } else { - types.emplace_back(substraitTypeToVeloxType(type)); - } } } catch (const VeloxException& err) { logValidateMsg( @@ -916,7 +933,7 @@ bool SubstraitToVeloxPlanValidator::validateAggRelFunctionType(const ::substrait err.message()); return false; } - auto funcName = SubstraitParser::mapToVeloxFunction(SubstraitParser::getSubFunctionName(funcSpec), isDecimal); + auto funcName = SubstraitParser::mapToVeloxFunction(SubstraitParser::getNameBeforeDelimiter(funcSpec), isDecimal); auto signaturesOpt = exec::getAggregateFunctionSignatures(funcName); if (!signaturesOpt) { logValidateMsg( @@ -1005,9 +1022,9 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::AggregateRel& ag const auto& aggFunction = smea.measure(); const auto& functionSpec = planConverter_.findFuncSpec(aggFunction.function_reference()); funcSpecs.emplace_back(functionSpec); - substraitTypeToVeloxType(aggFunction.output_type()); + SubstraitParser::parseType(aggFunction.output_type()); // Validate the size of arguments. - if (SubstraitParser::getSubFunctionName(functionSpec) == "count" && aggFunction.arguments().size() > 1) { + if (SubstraitParser::getNameBeforeDelimiter(functionSpec) == "count" && aggFunction.arguments().size() > 1) { logValidateMsg("native validation failed due to: count should have only one argument"); // Count accepts only one argument. return false; @@ -1043,6 +1060,10 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::AggregateRel& ag "min_merge", "max", "max_merge", + "min_by", + "min_by_merge", + "max_by", + "max_by_merge", "stddev_samp", "stddev_samp_merge", "stddev_pop", @@ -1075,7 +1096,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::AggregateRel& ag "approx_distinct"}; for (const auto& funcSpec : funcSpecs) { - auto funcName = SubstraitParser::getSubFunctionName(funcSpec); + auto funcName = SubstraitParser::getNameBeforeDelimiter(funcSpec); if (supportedAggFuncs.find(funcName) == supportedAggFuncs.end()) { logValidateMsg("native validation failed due to: " + funcName + " was not supported in AggregateRel."); return false; @@ -1117,11 +1138,7 @@ bool SubstraitToVeloxPlanValidator::validate(const ::substrait::ReadRel& readRel std::vector veloxTypeList; if (readRel.has_base_schema()) { const auto& baseSchema = readRel.base_schema(); - auto substraitTypeList = SubstraitParser::parseNamedStruct(baseSchema); - veloxTypeList.reserve(substraitTypeList.size()); - for (const auto& substraitType : substraitTypeList) { - veloxTypeList.emplace_back(toVeloxType(substraitType->type)); - } + veloxTypeList = SubstraitParser::parseNamedStruct(baseSchema); } int32_t inputPlanNodeId = 0; diff --git a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h index c723d4059bcc..d5d76a4dc1c1 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h +++ b/cpp/velox/substrait/SubstraitToVeloxPlanValidator.h @@ -18,6 +18,7 @@ #pragma once #include "SubstraitToVeloxPlan.h" +#include "velox/core/QueryCtx.h" namespace gluten { @@ -120,11 +121,8 @@ class SubstraitToVeloxPlanValidator { /// Validate Substrait literal. bool validateLiteral(const ::substrait::Expression_Literal& literal, const RowTypePtr& inputType); - /// Create RowType based on the type information in string. - TypePtr getRowType(const std::string& structType); - - /// Create DecimalType based on the type information in string. - TypePtr getDecimalType(const std::string& decimalType); + /// Validate Substrait if-then expression. + bool validateIfThen(const ::substrait::Expression_IfThen& ifThen, const RowTypePtr& inputType); /// Add necessary log for fallback void logValidateMsg(const std::string& log) { diff --git a/cpp/velox/substrait/TypeUtils.cc b/cpp/velox/substrait/TypeUtils.cc deleted file mode 100644 index c8d2f5fb17eb..000000000000 --- a/cpp/velox/substrait/TypeUtils.cc +++ /dev/null @@ -1,167 +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 "TypeUtils.h" -#include "SubstraitParser.h" -#include "velox/type/Type.h" - -namespace gluten { -std::vector getTypesFromCompoundName(std::string_view compoundName) { - // CompoundName is like ARRAY or MAP - // or ROW,ROW> - // the position of then delimiter is where the number of leftAngleBracket - // equals rightAngleBracket need to split. - std::vector types; - std::vector angleBracketNumEqualPos; - auto leftAngleBracketPos = compoundName.find("<"); - auto rightAngleBracketPos = compoundName.rfind(">"); - auto typesName = compoundName.substr(leftAngleBracketPos + 1, rightAngleBracketPos - leftAngleBracketPos - 1); - int leftAngleBracketNum = 0; - int rightAngleBracketNum = 0; - for (auto index = 0; index < typesName.length(); index++) { - if (typesName[index] == '<') { - leftAngleBracketNum++; - } - if (typesName[index] == '>') { - rightAngleBracketNum++; - } - if (typesName[index] == ',' && rightAngleBracketNum == leftAngleBracketNum) { - angleBracketNumEqualPos.push_back(index); - } - } - int startPos = 0; - for (auto delimeterPos : angleBracketNumEqualPos) { - types.emplace_back(typesName.substr(startPos, delimeterPos - startPos)); - startPos = delimeterPos + 1; - } - types.emplace_back(std::string_view(typesName.data() + startPos, typesName.length() - startPos)); - return types; -} - -// TODO Refactor using Bison. -std::string_view getNameBeforeDelimiter(const std::string& compoundName, const std::string& delimiter) { - std::size_t pos = compoundName.find(delimiter); - if (pos == std::string::npos) { - return compoundName; - } - return std::string_view(compoundName.data(), pos); -} - -std::pair getPrecisionAndScale(const std::string& typeName) { - std::size_t start = typeName.find_first_of("<"); - std::size_t end = typeName.find_last_of(">"); - if (start == std::string::npos || end == std::string::npos) { - throw std::runtime_error("Invalid decimal type."); - } - - std::string decimalType = typeName.substr(start + 1, end - start - 1); - std::size_t token_pos = decimalType.find_first_of(","); - auto precision = stoi(decimalType.substr(0, token_pos)); - auto scale = stoi(decimalType.substr(token_pos + 1, decimalType.length() - 1)); - return std::make_pair(precision, scale); -} - -TypePtr toVeloxType(const std::string& typeName, bool asLowerCase) { - VELOX_CHECK(!typeName.empty(), "Cannot convert empty string to Velox type."); - auto type = getNameBeforeDelimiter(typeName, "<"); - auto typeKind = mapNameToTypeKind(std::string(type)); - switch (typeKind) { - case TypeKind::BOOLEAN: - return BOOLEAN(); - case TypeKind::TINYINT: - return TINYINT(); - case TypeKind::SMALLINT: - return SMALLINT(); - case TypeKind::INTEGER: - return INTEGER(); - case TypeKind::BIGINT: - if (type == "SHORT_DECIMAL") { - auto decimal = getPrecisionAndScale(typeName); - return DECIMAL(decimal.first, decimal.second); - } else { - return BIGINT(); - } - case TypeKind::HUGEINT: { - auto decimal = getPrecisionAndScale(typeName); - return DECIMAL(decimal.first, decimal.second); - } - case TypeKind::REAL: - return REAL(); - case TypeKind::DOUBLE: - return DOUBLE(); - case TypeKind::VARCHAR: - return VARCHAR(); - case TypeKind::VARBINARY: - return VARBINARY(); - case TypeKind::ARRAY: { - auto fieldTypes = getTypesFromCompoundName(typeName); - VELOX_CHECK_EQ(fieldTypes.size(), 1, "The size of ARRAY type should be only one."); - return ARRAY(toVeloxType(std::string(fieldTypes[0]), asLowerCase)); - } - case TypeKind::MAP: { - auto fieldTypes = getTypesFromCompoundName(typeName); - VELOX_CHECK_EQ(fieldTypes.size(), 2, "The size of MAP type should be two."); - auto keyType = toVeloxType(std::string(fieldTypes[0]), asLowerCase); - auto valueType = toVeloxType(std::string(fieldTypes[1]), asLowerCase); - return MAP(keyType, valueType); - } - case TypeKind::ROW: { - auto fieldTypes = getTypesFromCompoundName(typeName); - VELOX_CHECK(!fieldTypes.empty(), "Converting empty ROW type from Substrait to Velox is not supported."); - - std::vector types; - std::vector names; - for (int idx = 0; idx < fieldTypes.size(); idx++) { - std::string fieldTypeAndName = std::string(fieldTypes[idx]); - size_t pos = fieldTypeAndName.find_last_of(':'); - if (pos == std::string::npos) { - // Name does not exist. - types.emplace_back(toVeloxType(fieldTypeAndName, asLowerCase)); - names.emplace_back("col_" + std::to_string(idx)); - } else { - types.emplace_back(toVeloxType(fieldTypeAndName.substr(0, pos), asLowerCase)); - std::string fieldName = fieldTypeAndName.substr(pos + 1, fieldTypeAndName.length()); - if (asLowerCase) { - folly::toLowerAscii(fieldName); - } - names.emplace_back(fieldName); - } - } - return ROW(std::move(names), std::move(types)); - } - case TypeKind::DATE: { - return DATE(); - } - case TypeKind::TIMESTAMP: { - return TIMESTAMP(); - } - case TypeKind::UNKNOWN: - return UNKNOWN(); - default: - VELOX_NYI("Velox type conversion not supported for type {}.", typeName); - } -} - -TypePtr substraitTypeToVeloxType(const std::string& substraitType) { - return toVeloxType(SubstraitParser::parseType(substraitType)); -} - -TypePtr substraitTypeToVeloxType(const ::substrait::Type& substraitType) { - return toVeloxType(SubstraitParser::parseType(substraitType).type); -} - -} // namespace gluten diff --git a/cpp/velox/substrait/TypeUtils.h b/cpp/velox/substrait/TypeUtils.h index ece68b6f1f19..b2aaf725789d 100644 --- a/cpp/velox/substrait/TypeUtils.h +++ b/cpp/velox/substrait/TypeUtils.h @@ -22,22 +22,6 @@ using namespace facebook::velox; namespace gluten { - -#ifndef TOVELOXTYPE_H -#define TOVELOXTYPE_H - -/// Return the Velox type according to the typename. -TypePtr toVeloxType(const std::string& typeName, bool asLowerCase = false); - -/// Return the Velox type according to substrait type string. -TypePtr substraitTypeToVeloxType(const std::string& substraitType); - -/// Return the Velox type according to substrait type. -TypePtr substraitTypeToVeloxType(const ::substrait::Type& substraitType); - -#endif /* TOVELOXTYPE_H */ - -std::string_view getNameBeforeDelimiter(const std::string& compoundName, const std::string& delimiter); #ifndef RANGETRAITS_H #define RANGETRAITS_H @@ -101,13 +85,6 @@ struct RangeTraits { using NativeType = std::string; }; -template <> -struct RangeTraits { - using RangeType = common::BigintRange; - using MultiRangeType = common::BigintMultiRange; - using NativeType = int32_t; -}; - template <> struct RangeTraits { using NativeType = int128_t; diff --git a/cpp/velox/substrait/VeloxSubstraitSignature.cc b/cpp/velox/substrait/VeloxSubstraitSignature.cc index 16897c4e5a03..ef1055f582cb 100644 --- a/cpp/velox/substrait/VeloxSubstraitSignature.cc +++ b/cpp/velox/substrait/VeloxSubstraitSignature.cc @@ -20,8 +20,12 @@ namespace gluten { -std::string VeloxSubstraitSignature::toSubstraitSignature(const TypeKind typeKind) { - switch (typeKind) { +std::string VeloxSubstraitSignature::toSubstraitSignature(const TypePtr& type) { + if (type->isDate()) { + return "date"; + } + + switch (type->kind()) { case TypeKind::BOOLEAN: return "bool"; case TypeKind::TINYINT: @@ -42,8 +46,6 @@ std::string VeloxSubstraitSignature::toSubstraitSignature(const TypeKind typeKin return "vbin"; case TypeKind::TIMESTAMP: return "ts"; - case TypeKind::DATE: - return "date"; case TypeKind::ARRAY: return "list"; case TypeKind::MAP: @@ -53,8 +55,108 @@ std::string VeloxSubstraitSignature::toSubstraitSignature(const TypeKind typeKin case TypeKind::UNKNOWN: return "u!name"; default: - VELOX_UNSUPPORTED("Substrait type signature conversion not supported for type {}.", mapTypeKindToName(typeKind)); + VELOX_UNSUPPORTED( + "Substrait type signature conversion not supported for type {}.", mapTypeKindToName(type->kind())); + } +} + +TypePtr VeloxSubstraitSignature::fromSubstraitSignature(const std::string& signature) { + if (signature == "bool") { + return BOOLEAN(); + } + + if (signature == "i8") { + return TINYINT(); + } + + if (signature == "i16") { + return SMALLINT(); + } + + if (signature == "i32") { + return INTEGER(); + } + + if (signature == "i64") { + return BIGINT(); + } + + if (signature == "fp32") { + return REAL(); + } + + if (signature == "fp64") { + return DOUBLE(); + } + + if (signature == "str") { + return VARCHAR(); + } + + if (signature == "vbin") { + return VARBINARY(); + } + + if (signature == "ts") { + return TIMESTAMP(); + } + + if (signature == "date") { + return DATE(); + } + + auto startWith = [](const std::string& str, const std::string& prefix) { + return str.size() >= prefix.size() && str.substr(0, prefix.size()) == prefix; + }; + + if (startWith(signature, "dec")) { + // Decimal type name is in the format of dec. + auto precisionStart = signature.find_first_of('<'); + auto tokenIndex = signature.find_first_of(','); + auto scaleEnd = signature.find_first_of('>'); + auto precision = stoi(signature.substr(precisionStart + 1, (tokenIndex - precisionStart - 1))); + auto scale = stoi(signature.substr(tokenIndex + 1, (scaleEnd - tokenIndex - 1))); + return DECIMAL(precision, scale); + } + + if (startWith(signature, "struct")) { + // Struct type name is in the format of struct. + auto structStart = signature.find_first_of('<'); + auto structEnd = signature.find_last_of('>'); + VELOX_CHECK( + structEnd - structStart > 1, "Native validation failed due to: more information is needed to create RowType"); + std::string childrenTypes = signature.substr(structStart + 1, structEnd - structStart - 1); + + // Split the types with delimiter. + std::string delimiter = ","; + std::size_t pos; + std::vector types; + std::vector names; + while ((pos = childrenTypes.find(delimiter)) != std::string::npos) { + auto typeStr = childrenTypes.substr(0, pos); + std::size_t endPos = pos; + if (startWith(typeStr, "dec") || startWith(typeStr, "struct")) { + endPos = childrenTypes.find(">") + 1; + if (endPos > pos) { + typeStr += childrenTypes.substr(pos, endPos - pos); + } else { + // For nested case, the end '>' could missing, + // so the last position is treated as end. + typeStr += childrenTypes.substr(pos); + endPos = childrenTypes.size(); + } + } + types.emplace_back(fromSubstraitSignature(typeStr)); + names.emplace_back(""); + childrenTypes.erase(0, endPos + delimiter.length()); + } + if (childrenTypes.size() > 0 && !startWith(childrenTypes, ">")) { + types.emplace_back(fromSubstraitSignature(childrenTypes)); + names.emplace_back(""); + } + return std::make_shared(std::move(names), std::move(types)); } + VELOX_UNSUPPORTED("Substrait type signature conversion to Velox type not supported for {}.", signature); } std::string VeloxSubstraitSignature::toSubstraitSignature( @@ -66,7 +168,7 @@ std::string VeloxSubstraitSignature::toSubstraitSignature( std::vector substraitTypeSignatures; substraitTypeSignatures.reserve(arguments.size()); for (const auto& type : arguments) { - substraitTypeSignatures.emplace_back(toSubstraitSignature(type->kind())); + substraitTypeSignatures.emplace_back(toSubstraitSignature(type)); } return functionName + ":" + folly::join("_", substraitTypeSignatures); } diff --git a/cpp/velox/substrait/VeloxSubstraitSignature.h b/cpp/velox/substrait/VeloxSubstraitSignature.h index c8bea816fd3f..8a54e4edcb6e 100644 --- a/cpp/velox/substrait/VeloxSubstraitSignature.h +++ b/cpp/velox/substrait/VeloxSubstraitSignature.h @@ -32,7 +32,10 @@ class VeloxSubstraitSignature { /// name>:__..._ for more /// detail information about the argument type please refer to link /// https://substrait.io/extensions/#function-signature-compound-names. - static std::string toSubstraitSignature(const TypeKind typeKind); + static std::string toSubstraitSignature(const TypePtr& type); + + /// Given a substrait type signature, return the Velox type. + static TypePtr fromSubstraitSignature(const std::string& signature); /// Given a velox scalar function name and argument types, return the /// substrait function signature. diff --git a/cpp/velox/substrait/VeloxToSubstraitExpr.cc b/cpp/velox/substrait/VeloxToSubstraitExpr.cc index d2b7f2e0fd1c..4cd504929307 100644 --- a/cpp/velox/substrait/VeloxToSubstraitExpr.cc +++ b/cpp/velox/substrait/VeloxToSubstraitExpr.cc @@ -139,10 +139,6 @@ const ::substrait::Expression_Literal& toSubstraitNotNullLiteral( literalExpr->set_timestamp(micros); break; } - case velox::TypeKind::DATE: { - literalExpr->set_date(variantValue.value().days()); - break; - } case velox::TypeKind::VARCHAR: { auto vCharValue = variantValue.value(); ::substrait::Expression_Literal::VarChar* sVarChar = new ::substrait::Expression_Literal::VarChar(); @@ -247,17 +243,6 @@ const ::substrait::Expression_Literal& toSubstraitNotNullLiteral -const ::substrait::Expression_Literal& toSubstraitNotNullLiteral( - google::protobuf::Arena& arena, - const Date& value) { - ::substrait::Expression_Literal* literalExpr = - google::protobuf::Arena::CreateMessage<::substrait::Expression_Literal>(&arena); - literalExpr->set_date(value.days()); - literalExpr->set_nullable(false); - return *literalExpr; -} - template <> const ::substrait::Expression_Literal& toSubstraitNotNullLiteral( google::protobuf::Arena& arena, @@ -373,6 +358,12 @@ const ::substrait::Expression& VeloxToSubstraitExprConvertor::toSubstraitExpr( return *substraitExpr; } + if (auto derefExpr = std::dynamic_pointer_cast(expr)) { + substraitExpr->mutable_selection()->MergeFrom(toSubstraitExpr(arena, derefExpr, inputType)); + + return *substraitExpr; + } + VELOX_UNSUPPORTED("Unsupport Expr '{}' in Substrait", expr->toString()); } @@ -441,6 +432,35 @@ const ::substrait::Expression_FieldReference& VeloxToSubstraitExprConvertor::toS return *substraitFieldExpr; } +const ::substrait::Expression_FieldReference& VeloxToSubstraitExprConvertor::toSubstraitExpr( + google::protobuf::Arena& arena, + const std::shared_ptr& derefExpr, + const RowTypePtr& inputType) { + ::substrait::Expression_FieldReference* substraitFieldExpr = + google::protobuf::Arena::CreateMessage<::substrait::Expression_FieldReference>(&arena); + + ::substrait::Expression_ReferenceSegment_StructField* directStruct = + substraitFieldExpr->mutable_direct_reference()->mutable_struct_field(); + + auto input = toSubstraitExpr(arena, derefExpr->inputs()[0], inputType); + VELOX_USER_CHECK(input.has_selection(), "Non-field expression is not supported"); + auto inputRef = input.selection().direct_reference(); + + ::substrait::Expression_ReferenceSegment_StructField* childStruct = + google::protobuf::Arena::CreateMessage<::substrait::Expression_ReferenceSegment_StructField>(&arena); + ::substrait::Expression_ReferenceSegment* refSegment = + google::protobuf::Arena::CreateMessage<::substrait::Expression_ReferenceSegment>(&arena); + directStruct->MergeFrom(inputRef.struct_field()); + childStruct->set_field(derefExpr->index()); + refSegment->set_allocated_struct_field(childStruct); + ::substrait::Expression_ReferenceSegment_StructField* innerChild = directStruct; + while (innerChild->has_child()) { + innerChild = innerChild->mutable_child()->mutable_struct_field(); + } + innerChild->set_allocated_child(refSegment); + return *substraitFieldExpr; +} + const ::substrait::Expression& VeloxToSubstraitExprConvertor::toSubstraitExpr( google::protobuf::Arena& arena, const std::shared_ptr& callTypeExpr, diff --git a/cpp/velox/substrait/VeloxToSubstraitExpr.h b/cpp/velox/substrait/VeloxToSubstraitExpr.h index 392275f8505b..d60132ccc227 100644 --- a/cpp/velox/substrait/VeloxToSubstraitExpr.h +++ b/cpp/velox/substrait/VeloxToSubstraitExpr.h @@ -59,6 +59,11 @@ class VeloxToSubstraitExprConvertor { const std::shared_ptr& fieldExpr, const RowTypePtr& inputType); + const ::substrait::Expression_FieldReference& toSubstraitExpr( + google::protobuf::Arena& arena, + const std::shared_ptr& derefExpr, + const RowTypePtr& inputType); + /// Convert Velox vector to Substrait literal. const ::substrait::Expression_Literal& toSubstraitLiteral( google::protobuf::Arena& arena, diff --git a/cpp/velox/substrait/VeloxToSubstraitPlan.cc b/cpp/velox/substrait/VeloxToSubstraitPlan.cc index ca1b0c607d64..409b85498ddc 100644 --- a/cpp/velox/substrait/VeloxToSubstraitPlan.cc +++ b/cpp/velox/substrait/VeloxToSubstraitPlan.cc @@ -260,7 +260,8 @@ void VeloxToSubstraitPlanConvertor::toSubstrait( } } - auto referenceNumber = extensionCollector_->getReferenceNumber(funName, arguments, aggregateNode->step()); + auto referenceNumber = + extensionCollector_->getReferenceNumber(funName, aggregate.rawInputTypes, aggregateNode->step()); aggFunction->set_function_reference(referenceNumber); diff --git a/cpp/velox/substrait/VeloxToSubstraitType.cc b/cpp/velox/substrait/VeloxToSubstraitType.cc index a00cc082a725..b08fe83db6a1 100644 --- a/cpp/velox/substrait/VeloxToSubstraitType.cc +++ b/cpp/velox/substrait/VeloxToSubstraitType.cc @@ -25,6 +25,13 @@ const ::substrait::Type& VeloxToSubstraitTypeConvertor::toSubstraitType( google::protobuf::Arena& arena, const velox::TypePtr& type) { ::substrait::Type* substraitType = google::protobuf::Arena::CreateMessage<::substrait::Type>(&arena); + if (type->isDate()) { + auto substraitDate = google::protobuf::Arena::CreateMessage<::substrait::Type_Date>(&arena); + substraitDate->set_nullability(::substrait::Type_Nullability_NULLABILITY_NULLABLE); + substraitType->set_allocated_date(substraitDate); + return *substraitType; + } + switch (type->kind()) { case velox::TypeKind::BOOLEAN: { auto substraitBool = google::protobuf::Arena::CreateMessage<::substrait::Type_Boolean>(&arena); @@ -119,12 +126,6 @@ const ::substrait::Type& VeloxToSubstraitTypeConvertor::toSubstraitType( substraitType->set_allocated_user_defined(substraitUserDefined); break; } - case velox::TypeKind::DATE: { - auto substraitDate = google::protobuf::Arena::CreateMessage<::substrait::Type_Date>(&arena); - substraitDate->set_nullability(::substrait::Type_Nullability_NULLABILITY_NULLABLE); - substraitType->set_allocated_date(substraitDate); - break; - } case velox::TypeKind::FUNCTION: case velox::TypeKind::OPAQUE: case velox::TypeKind::INVALID: diff --git a/cpp/velox/tests/CMakeLists.txt b/cpp/velox/tests/CMakeLists.txt index 9dde3d6f4c05..c2d412a56b0b 100644 --- a/cpp/velox/tests/CMakeLists.txt +++ b/cpp/velox/tests/CMakeLists.txt @@ -13,7 +13,6 @@ # See the License for the specific language governing permissions and # limitations under the License. -find_arrow_lib(${PARQUET_LIB_NAME}) function(add_velox_test TEST_EXEC) set(options) set(one_value_args) @@ -33,12 +32,13 @@ function(add_velox_test TEST_EXEC) endif() add_executable(${TEST_EXEC} ${SOURCES}) target_include_directories(${TEST_EXEC} PRIVATE ${CMAKE_SOURCE_DIR}/velox ${CMAKE_SOURCE_DIR}/src) - target_link_libraries(${TEST_EXEC} gflags Arrow::parquet velox GTest::gtest GTest::gtest_main glog::glog benchmark::benchmark simdjson) - gtest_discover_tests(${TEST_EXEC}) + target_link_libraries(${TEST_EXEC} velox GTest::gtest GTest::gtest_main google::glog benchmark::benchmark simdjson) + gtest_discover_tests(${TEST_EXEC} DISCOVERY_MODE PRE_TEST) endfunction() add_velox_test(velox_shuffle_writer_test SOURCES VeloxShuffleWriterTest.cc) -add_velox_test(orc_test SOURCES OrcTest.cc) +# TODO: ORC is not well supported. +# add_velox_test(orc_test SOURCES OrcTest.cc) add_velox_test(velox_operators_test SOURCES VeloxColumnarToRowTest.cc VeloxRowToColumnarTest.cc VeloxColumnarBatchSerializerTest.cc) add_velox_test( velox_plan_conversion_test @@ -54,4 +54,4 @@ add_velox_test( JsonToProtoConverter.cc FilePathGenerator.cc) add_velox_test(spark_functions_test SOURCES SparkFunctionTest.cc) -add_velox_test(execution_ctx_test SOURCES ExecutionCtxTest.cc) +add_velox_test(execution_ctx_test SOURCES RuntimeTest.cc) diff --git a/cpp/velox/tests/ExecutionCtxTest.cc b/cpp/velox/tests/ExecutionCtxTest.cc deleted file mode 100644 index 8242135c545d..000000000000 --- a/cpp/velox/tests/ExecutionCtxTest.cc +++ /dev/null @@ -1,175 +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 "compute/VeloxExecutionCtx.h" - -#include - -namespace gluten { - -class DummyExecutionCtx final : public ExecutionCtx { - public: - DummyExecutionCtx(const std::unordered_map& conf) : ExecutionCtx(conf) {} - - ResourceHandle createResultIterator( - MemoryManager* memoryManager, - const std::string& spillDir, - const std::vector>& inputs, - const std::unordered_map& sessionConf) override { - auto resIter = std::make_unique(); - auto iter = std::make_shared(std::move(resIter)); - return resultIteratorHolder_.insert(iter); - } - MemoryManager* createMemoryManager( - const std::string& name, - std::shared_ptr ptr, - std::unique_ptr uniquePtr) override { - return nullptr; - } - ResourceHandle addResultIterator(std::shared_ptr ptr) override { - return kInvalidResourceHandle; - } - std::shared_ptr getResultIterator(ResourceHandle handle) override { - return resultIteratorHolder_.lookup(handle); - } - void releaseResultIterator(ResourceHandle handle) override {} - ResourceHandle addBatch(std::shared_ptr ptr) override { - return kInvalidResourceHandle; - } - std::shared_ptr getBatch(ResourceHandle handle) override { - return std::shared_ptr(); - } - ResourceHandle createOrGetEmptySchemaBatch(int32_t numRows) override { - return kInvalidResourceHandle; - } - void releaseBatch(ResourceHandle handle) override {} - ResourceHandle createColumnar2RowConverter(MemoryManager* memoryManager) override { - return kInvalidResourceHandle; - } - std::shared_ptr getColumnar2RowConverter(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseColumnar2RowConverter(ResourceHandle handle) override {} - ResourceHandle createRow2ColumnarConverter(MemoryManager* memoryManager, struct ArrowSchema* cSchema) override { - return kInvalidResourceHandle; - } - std::shared_ptr getRow2ColumnarConverter(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseRow2ColumnarConverter(ResourceHandle handle) override {} - ResourceHandle createShuffleWriter( - int numPartitions, - std::shared_ptr partitionWriterCreator, - const ShuffleWriterOptions& options, - MemoryManager* memoryManager) override { - return kInvalidResourceHandle; - } - std::shared_ptr getShuffleWriter(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseShuffleWriter(ResourceHandle handle) override {} - Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) override { - static Metrics m(1); - return &m; - } - ResourceHandle createDatasource( - const std::string& filePath, - MemoryManager* memoryManager, - std::shared_ptr schema) override { - return kInvalidResourceHandle; - } - std::shared_ptr getDatasource(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseDatasource(ResourceHandle handle) override {} - ResourceHandle createShuffleReader( - std::shared_ptr schema, - ReaderOptions options, - arrow::MemoryPool* pool, - MemoryManager* memoryManager) override { - return kInvalidResourceHandle; - } - std::shared_ptr getShuffleReader(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseShuffleReader(ResourceHandle handle) override {} - ResourceHandle createColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) override { - return kInvalidResourceHandle; - } - std::unique_ptr createTempColumnarBatchSerializer( - MemoryManager* memoryManager, - arrow::MemoryPool* arrowPool, - struct ArrowSchema* cSchema) override { - return std::unique_ptr(); - } - std::shared_ptr getColumnarBatchSerializer(ResourceHandle handle) override { - return std::shared_ptr(); - } - void releaseColumnarBatchSerializer(ResourceHandle handle) override {} - ResourceHandle select(MemoryManager*, ResourceHandle, std::vector) override { - return kInvalidResourceHandle; - } - - private: - ResourceMap> resultIteratorHolder_; - - class DummyResultIterator : public ColumnarBatchIterator { - public: - std::shared_ptr next() override { - if (!hasNext_) { - return nullptr; - } - hasNext_ = false; - - auto fArrInt32 = arrow::field("f_int32", arrow::int32()); - auto rbSchema = arrow::schema({fArrInt32}); - auto rb = arrow::RecordBatch::Make(rbSchema, 1, std::vector>{}); - return std::make_shared(rb); - } - - private: - bool hasNext_ = true; - }; -}; - -static ExecutionCtx* DummyExecutionCtxFactory(const std::unordered_map conf) { - return new DummyExecutionCtx(conf); -} - -TEST(TestExecutionCtx, CreateExecutionCtx) { - ExecutionCtx::registerFactory("DUMMY", DummyExecutionCtxFactory); - auto executionCtx = ExecutionCtx::create("DUMMY"); - ASSERT_EQ(typeid(*executionCtx), typeid(DummyExecutionCtx)); - ExecutionCtx::release(executionCtx); -} - -TEST(TestExecutionCtx, GetResultIterator) { - auto executionCtx = std::make_shared(std::unordered_map()); - auto handle = executionCtx->createResultIterator(nullptr, "/tmp/test-spill", {}, {}); - auto iter = executionCtx->getResultIterator(handle); - ASSERT_TRUE(iter->hasNext()); - auto next = iter->next(); - ASSERT_NE(next, nullptr); - ASSERT_FALSE(iter->hasNext()); - next = iter->next(); - ASSERT_EQ(next, nullptr); -} - -} // namespace gluten diff --git a/cpp/velox/tests/FunctionTest.cc b/cpp/velox/tests/FunctionTest.cc index 8baf46febc4b..a5d2b5fade3d 100644 --- a/cpp/velox/tests/FunctionTest.cc +++ b/cpp/velox/tests/FunctionTest.cc @@ -28,6 +28,10 @@ #include "FilePathGenerator.h" +#include "velox/core/QueryCtx.h" + +#include "substrait/SubstraitParser.h" + using namespace facebook::velox; using namespace facebook::velox::test; @@ -70,15 +74,15 @@ TEST_F(FunctionTest, getIdxFromNodeName) { TEST_F(FunctionTest, getNameBeforeDelimiter) { std::string functionSpec = "lte:fp64_fp64"; - std::string_view funcName = getNameBeforeDelimiter(functionSpec, ":"); + std::string_view funcName = SubstraitParser::getNameBeforeDelimiter(functionSpec); ASSERT_EQ(funcName, "lte"); functionSpec = "lte:"; - funcName = getNameBeforeDelimiter(functionSpec, ":"); + funcName = SubstraitParser::getNameBeforeDelimiter(functionSpec); ASSERT_EQ(funcName, "lte"); functionSpec = "lte"; - funcName = getNameBeforeDelimiter(functionSpec, ":"); + funcName = SubstraitParser::getNameBeforeDelimiter(functionSpec); ASSERT_EQ(funcName, "lte"); } @@ -167,9 +171,9 @@ TEST_F(FunctionTest, setVectorFromVariants) { ASSERT_EQ("1970-01-01T02:30:20.000000000", resultVec->asFlatVector()->valueAt(0).toString()); ASSERT_EQ("1970-01-01T02:27:55.000000000", resultVec->asFlatVector()->valueAt(1).toString()); - resultVec = setVectorFromVariants(DATE(), {variant(Date(9020)), variant(Date(8875))}, pool_.get()); - ASSERT_EQ("1994-09-12", resultVec->asFlatVector()->valueAt(0).toString()); - ASSERT_EQ("1994-04-20", resultVec->asFlatVector()->valueAt(1).toString()); + resultVec = setVectorFromVariants(DATE(), {variant(9020), variant(8875)}, pool_.get()); + ASSERT_EQ("1994-09-12", DATE()->toString(resultVec->asFlatVector()->valueAt(0))); + ASSERT_EQ("1994-04-20", DATE()->toString(resultVec->asFlatVector()->valueAt(1))); resultVec = setVectorFromVariants(INTERVAL_DAY_TIME(), {variant(9020LL), variant(8875LL)}, pool_.get()); ASSERT_TRUE(resultVec->type()->isIntervalDayTime()); @@ -178,18 +182,36 @@ TEST_F(FunctionTest, setVectorFromVariants) { } TEST_F(FunctionTest, getFunctionType) { - std::vector types; - SubstraitParser::getSubFunctionTypes("sum:opt_i32", types); + std::vector types = SubstraitParser::getSubFunctionTypes("sum:opt_i32"); ASSERT_EQ("i32", types[0]); - types.clear(); - SubstraitParser::getSubFunctionTypes("sum:i32", types); + types = SubstraitParser::getSubFunctionTypes("sum:i32"); ASSERT_EQ("i32", types[0]); - types.clear(); - SubstraitParser::getSubFunctionTypes("sum:opt_str_str", types); + types = SubstraitParser::getSubFunctionTypes("sum:opt_str_str"); ASSERT_EQ(2, types.size()); ASSERT_EQ("str", types[0]); ASSERT_EQ("str", types[1]); } + +TEST_F(FunctionTest, sigToTypes) { + std::vector types = SubstraitParser::sigToTypes("sum:opt_i32"); + ASSERT_EQ(types[0]->kind(), TypeKind::INTEGER); + + types = SubstraitParser::sigToTypes("and:opt_bool_bool"); + ASSERT_EQ(2, types.size()); + ASSERT_EQ(types[0]->kind(), TypeKind::BOOLEAN); + ASSERT_EQ(types[1]->kind(), TypeKind::BOOLEAN); + + types = SubstraitParser::sigToTypes("sum:dec<12,9>"); + ASSERT_EQ(getDecimalPrecisionScale(*types[0]).first, 12); + ASSERT_EQ(getDecimalPrecisionScale(*types[0]).second, 9); + + types = SubstraitParser::sigToTypes("sum:struct,bool>"); + ASSERT_EQ(types[0]->kind(), TypeKind::ROW); + ASSERT_EQ(types[0]->childAt(0)->kind(), TypeKind::INTEGER); + ASSERT_EQ(types[0]->childAt(1)->kind(), TypeKind::VARCHAR); + ASSERT_TRUE(types[0]->childAt(2)->isDecimal()); + ASSERT_EQ(types[0]->childAt(3)->kind(), TypeKind::BOOLEAN); +} } // namespace gluten diff --git a/cpp/velox/tests/OrcTest.cc b/cpp/velox/tests/OrcTest.cc index 29d4fd3667a5..0861701cf060 100644 --- a/cpp/velox/tests/OrcTest.cc +++ b/cpp/velox/tests/OrcTest.cc @@ -17,7 +17,7 @@ #include "arrow/c/abi.h" #include "benchmarks/BatchStreamIterator.h" -#include "benchmarks/BenchmarkUtils.h" +#include "benchmarks/common/BenchmarkUtils.h" #include "utils/TestUtils.h" diff --git a/cpp/velox/tests/RuntimeTest.cc b/cpp/velox/tests/RuntimeTest.cc new file mode 100644 index 000000000000..c6af63b8e0c2 --- /dev/null +++ b/cpp/velox/tests/RuntimeTest.cc @@ -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. + */ + +#include "compute/VeloxRuntime.h" + +#include + +namespace gluten { + +class DummyRuntime final : public Runtime { + public: + DummyRuntime(const std::unordered_map& conf) : Runtime(conf) {} + + void parsePlan(const uint8_t* data, int32_t size, SparkTaskInfo taskInfo) override {} + + std::shared_ptr createResultIterator( + MemoryManager* memoryManager, + const std::string& spillDir, + const std::vector>& inputs, + const std::unordered_map& sessionConf) override { + auto resIter = std::make_unique(); + auto iter = std::make_shared(std::move(resIter)); + return iter; + } + MemoryManager* createMemoryManager( + const std::string& name, + std::shared_ptr ptr, + std::unique_ptr uniquePtr) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr createOrGetEmptySchemaBatch(int32_t numRows) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr createColumnar2RowConverter(MemoryManager* memoryManager) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr createRow2ColumnarConverter( + MemoryManager* memoryManager, + struct ArrowSchema* cSchema) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr createShuffleWriter( + int numPartitions, + std::shared_ptr partitionWriterCreator, + const ShuffleWriterOptions& options, + MemoryManager* memoryManager) override { + throw GlutenException("Not yet implemented"); + } + Metrics* getMetrics(ColumnarBatchIterator* rawIter, int64_t exportNanos) override { + static Metrics m(1); + return &m; + } + std::shared_ptr createDatasource( + const std::string& filePath, + MemoryManager* memoryManager, + std::shared_ptr schema) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr createShuffleReader( + std::shared_ptr schema, + ShuffleReaderOptions options, + arrow::MemoryPool* pool, + MemoryManager* memoryManager) override { + throw GlutenException("Not yet implemented"); + } + std::unique_ptr createColumnarBatchSerializer( + MemoryManager* memoryManager, + arrow::MemoryPool* arrowPool, + struct ArrowSchema* cSchema) override { + throw GlutenException("Not yet implemented"); + } + std::shared_ptr select(MemoryManager*, std::shared_ptr, std::vector) override { + throw GlutenException("Not yet implemented"); + } + std::string planString(bool details, const std::unordered_map& sessionConf) override { + throw GlutenException("Not yet implemented"); + } + + private: + ResourceMap> resultIteratorHolder_; + + class DummyResultIterator : public ColumnarBatchIterator { + public: + std::shared_ptr next() override { + if (!hasNext_) { + return nullptr; + } + hasNext_ = false; + + auto fArrInt32 = arrow::field("f_int32", arrow::int32()); + auto rbSchema = arrow::schema({fArrInt32}); + auto rb = arrow::RecordBatch::Make(rbSchema, 1, std::vector>{}); + return std::make_shared(rb); + } + + private: + bool hasNext_ = true; + }; +}; + +static Runtime* dummyRuntimeFactory(const std::unordered_map conf) { + return new DummyRuntime(conf); +} + +TEST(TestRuntime, CreateRuntime) { + Runtime::registerFactory("DUMMY", dummyRuntimeFactory); + auto runtime = Runtime::create("DUMMY"); + ASSERT_EQ(typeid(*runtime), typeid(DummyRuntime)); + Runtime::release(runtime); +} + +TEST(TestRuntime, GetResultIterator) { + auto runtime = std::make_shared(std::unordered_map()); + auto iter = runtime->createResultIterator(nullptr, "/tmp/test-spill", {}, {}); + ASSERT_TRUE(iter->hasNext()); + auto next = iter->next(); + ASSERT_NE(next, nullptr); + ASSERT_FALSE(iter->hasNext()); + next = iter->next(); + ASSERT_EQ(next, nullptr); +} + +} // namespace gluten diff --git a/cpp/velox/tests/SparkFunctionTest.cc b/cpp/velox/tests/SparkFunctionTest.cc index 06e643561951..a340b4fcacbb 100644 --- a/cpp/velox/tests/SparkFunctionTest.cc +++ b/cpp/velox/tests/SparkFunctionTest.cc @@ -25,7 +25,7 @@ using namespace facebook::velox; class SparkFunctionTest : public SparkFunctionBaseTest { public: SparkFunctionTest() { - gluten::registerFunctionOverwrite(); + gluten::registerAllFunctions(); } protected: diff --git a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc index ce54d06d119f..86a93583aa0a 100644 --- a/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc +++ b/cpp/velox/tests/VeloxColumnarBatchSerializerTest.cc @@ -43,8 +43,8 @@ TEST_F(VeloxColumnarBatchSerializerTest, serialize) { makeNullableFlatVector({std::nullopt, true, false, std::nullopt, true}), makeFlatVector({"alice0", "bob1", "alice2", "bob3", "Alice4uuudeuhdhfudhfudhfudhbvudubvudfvu"}), makeNullableFlatVector({"alice", "bob", std::nullopt, std::nullopt, "Alice"}), - makeShortDecimalFlatVector({34567235, 4567, 222, 34567, 333}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({34567235, 4567, 222, 34567, 333}, DECIMAL(20, 4)), + makeNullableFlatVector({34567235, 4567, 222, 34567, 333}, DECIMAL(12, 4)), + makeNullableFlatVector({34567235, 4567, 222, 34567, 333}, DECIMAL(20, 4)), }; auto vector = makeRowVector(children); auto batch = std::make_shared(vector); diff --git a/cpp/velox/tests/VeloxShuffleWriterTest.cc b/cpp/velox/tests/VeloxShuffleWriterTest.cc index 15c64bc5d2b8..54d64beb2235 100644 --- a/cpp/velox/tests/VeloxShuffleWriterTest.cc +++ b/cpp/velox/tests/VeloxShuffleWriterTest.cc @@ -16,25 +16,15 @@ */ #include -#include -#include #include -#include -#include -#include -#include -#include - -#include "memory/VeloxColumnarBatch.h" -#include "memory/VeloxMemoryManager.h" + #include "shuffle/LocalPartitionWriter.h" -#include "shuffle/Utils.h" -#include "shuffle/VeloxShuffleReader.h" #include "shuffle/VeloxShuffleWriter.h" #include "shuffle/rss/CelebornPartitionWriter.h" -#include "shuffle/rss/RssClient.h" #include "utils/TestUtils.h" #include "utils/VeloxArrowUtils.h" +#include "utils/tests/MemoryPoolUtils.h" +#include "utils/tests/VeloxShuffleWriterTestBase.h" #include "velox/vector/tests/utils/VectorTestBase.h" using namespace facebook; @@ -45,415 +35,104 @@ using namespace arrow::ipc; namespace gluten { namespace { -arrow::Status splitRowVectorStatus(VeloxShuffleWriter& shuffleWriter, velox::RowVectorPtr vector) { - std::shared_ptr cb = std::make_shared(vector); - return shuffleWriter.split(cb, ShuffleWriter::kMinMemLimit); -} -} // namespace -struct ShuffleTestParams { - PartitionWriterType partition_writer_type; - arrow::Compression::type compression_type; - CompressionMode compression_mode; - - std::string toString() const { - std::ostringstream out; - out << "partition_writer_type = " << partition_writer_type << "compression_type = " << compression_type - << ", compression_mode = " << compression_mode; - return out.str(); +facebook::velox::RowVectorPtr takeRows(const facebook::velox::RowVectorPtr& source, const std::vector& idxs) { + facebook::velox::RowVectorPtr copy = facebook::velox::RowVector::createEmpty(source->type(), source->pool()); + for (int32_t idx : idxs) { + copy->append(source->slice(idx, 1).get()); } -}; - -class LocalRssClient : public RssClient { - public: - LocalRssClient(std::string dataFile) : dataFile_(dataFile) {} - - int32_t pushPartitionData(int32_t partitionId, char* bytes, int64_t size) { - auto idx = -1; - auto maybeIdx = partitionIdx_.find(partitionId); - auto returnSize = size; - if (maybeIdx == partitionIdx_.end()) { - idx = partitionIdx_.size(); - partitionIdx_[partitionId] = idx; - auto buffer = arrow::AllocateResizableBuffer(0).ValueOrDie(); - partitionBuffers_.push_back(std::move(buffer)); - // Add EOS length. - returnSize += sizeof(int32_t) * 2; - } else { - idx = maybeIdx->second; - } + return copy; +} - auto& buffer = partitionBuffers_[idx]; - auto newSize = buffer->size() + size; - if (buffer->capacity() < newSize) { - ASSERT_NOT_OK(buffer->Reserve(newSize)); - } - memcpy(buffer->mutable_data() + buffer->size(), bytes, size); - ASSERT_NOT_OK(buffer->Resize(newSize)); - return returnSize; - } +std::vector createShuffleTestParams() { + std::vector params; - void stop() { - std::shared_ptr fout; - ARROW_ASSIGN_OR_THROW(fout, arrow::io::FileOutputStream::Open(dataFile_, true)); + std::vector writerTypes = {PartitionWriterType::kLocal, PartitionWriterType::kCeleborn}; - for (auto item : partitionIdx_) { - auto idx = item.second; - ASSERT_NOT_OK(fout->Write(partitionBuffers_[idx]->data(), partitionBuffers_[idx]->size())); - ASSERT_NOT_OK(writeEos(fout.get())); - ASSERT_NOT_OK(fout->Flush()); - } - ASSERT_NOT_OK(fout->Close()); - } + std::vector compressions = { + arrow::Compression::UNCOMPRESSED, arrow::Compression::LZ4_FRAME, arrow::Compression::ZSTD}; - private: - std::string dataFile_; - std::vector> partitionBuffers_; - std::map partitionIdx_; -}; - -class VeloxShuffleWriterTest : public ::testing::TestWithParam, public velox::test::VectorTestBase { - protected: - void SetUp() override { - const std::string tmpDirPrefix = "columnar-shuffle-test"; - ARROW_ASSIGN_OR_THROW(tmpDir1_, arrow::internal::TemporaryDir::Make(tmpDirPrefix)) - ARROW_ASSIGN_OR_THROW(tmpDir2_, arrow::internal::TemporaryDir::Make(tmpDirPrefix)) - auto configDirs = tmpDir1_->path().ToString() + "," + tmpDir2_->path().ToString(); - - setenv(kGlutenSparkLocalDirs.c_str(), configDirs.c_str(), 1); - - shuffleWriterOptions_ = ShuffleWriterOptions::defaults(); - shuffleWriterOptions_.buffer_compress_threshold = 0; - shuffleWriterOptions_.memory_pool = arrowPool_.get(); - - ShuffleTestParams params = GetParam(); - if (params.partition_writer_type == PartitionWriterType::kCeleborn) { - auto configuredDirs = getConfiguredLocalDirs().ValueOrDie(); - auto dataFile = createTempShuffleFile(configuredDirs[0]).ValueOrDie(); - shuffleWriterOptions_.data_file = dataFile; - partitionWriterCreator_ = - std::make_shared(std::make_shared(dataFile)); - shuffleWriterOptions_.partition_writer_type = kCeleborn; - } else { - partitionWriterCreator_ = std::make_shared(); - } - shuffleWriterOptions_.compression_type = params.compression_type; - shuffleWriterOptions_.compression_mode = params.compression_mode; - - std::vector children1 = { - makeNullableFlatVector({1, 2, 3, std::nullopt, 4, std::nullopt, 5, 6, std::nullopt, 7}), - makeNullableFlatVector({1, -1, std::nullopt, std::nullopt, -2, 2, std::nullopt, std::nullopt, 3, -3}), - makeNullableFlatVector({1, 2, 3, 4, std::nullopt, 5, 6, 7, 8, std::nullopt}), - makeNullableFlatVector( - {std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt, - std::nullopt}), - makeNullableFlatVector( - {-0.1234567, - std::nullopt, - 0.1234567, - std::nullopt, - -0.142857, - std::nullopt, - 0.142857, - 0.285714, - 0.428617, - std::nullopt}), - makeNullableFlatVector( - {std::nullopt, true, false, std::nullopt, true, true, false, true, std::nullopt, std::nullopt}), - makeFlatVector( - {"alice0", "bob1", "alice2", "bob3", "Alice4", "Bob5", "AlicE6", "boB7", "ALICE8", "BOB9"}), - makeNullableFlatVector( - {"alice", "bob", std::nullopt, std::nullopt, "Alice", "Bob", std::nullopt, "alicE", std::nullopt, "boB"}), - }; - inputVector1_ = makeRowVector(children1); - children1.insert((children1.begin()), makeFlatVector({1, 2, 2, 2, 2, 1, 1, 1, 2, 1})); - hashInputVector1_ = makeRowVector(children1); - - std::vector children2 = { - makeNullableFlatVector({std::nullopt, std::nullopt}), - makeFlatVector({1, -1}), - makeNullableFlatVector({100, std::nullopt}), - makeFlatVector({1, 1}), - makeFlatVector({0.142857, -0.142857}), - makeFlatVector({true, false}), - makeFlatVector( - {"bob", - "alicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealice"}), - makeNullableFlatVector({std::nullopt, std::nullopt}), - }; - inputVector2_ = makeRowVector(children2); - children2.insert((children2.begin()), makeFlatVector({2, 2})); - hashInputVector2_ = makeRowVector(children2); - } + std::vector modes = {CompressionMode::BUFFER, CompressionMode::ROWVECTOR}; - void TearDown() override { - if (file_ != nullptr && !file_->closed()) { - GLUTEN_THROW_NOT_OK(file_->Close()); + for (const auto& writerType : writerTypes) { + for (const auto& compression : compressions) { + for (const auto& mode : modes) { + params.push_back(ShuffleTestParams{writerType, compression, mode}); + } } } - static void checkFileExists(const std::string& fileName) { - ASSERT_EQ(*arrow::internal::FileExists(*arrow::internal::PlatformFilename::FromString(fileName)), true); - } - - void splitRowVector(VeloxShuffleWriter& shuffleWriter, velox::RowVectorPtr vector) { - std::shared_ptr cb = std::make_shared(vector); - GLUTEN_THROW_NOT_OK(shuffleWriter.split(cb, ShuffleWriter::kMinMemLimit)); - } - - RowVectorPtr takeRows(const RowVectorPtr& source, const std::vector& idxs) const { - RowVectorPtr copy = RowVector::createEmpty(source->type(), source->pool()); - for (int32_t idx : idxs) { - copy->append(source->slice(idx, 1).get()); - } - return copy; - } + return params; +} - std::shared_ptr getArrowSchema(velox::RowVectorPtr& rowVector) { - return toArrowSchema(rowVector->type(), pool()); - } +static const auto kShuffleWriteTestParams = createShuffleTestParams(); - void setReadableFile(const std::string& fileName) { - if (file_ != nullptr && !file_->closed()) { - GLUTEN_THROW_NOT_OK(file_->Close()); - } - GLUTEN_ASSIGN_OR_THROW(file_, arrow::io::ReadableFile::Open(fileName)) - } +} // namespace - void getRowVectors(std::shared_ptr schema, std::vector& vectors) { - ReaderOptions options; - options.compression_type = shuffleWriterOptions_.compression_type; - options.compression_mode = shuffleWriterOptions_.compression_mode; - auto reader = std::make_shared(schema, options, arrowPool_.get(), pool_); - auto iter = reader->readStream(file_); - while (iter->hasNext()) { - auto vector = std::dynamic_pointer_cast(iter->next())->getRowVector(); - vectors.emplace_back(vector); - } +TEST_P(SinglePartitioningShuffleWriter, single) { + // Split 1 RowVector. + { + auto shuffleWriter = createShuffleWriter(); + testShuffleWrite(*shuffleWriter, {inputVector1_}); } - - // 1 partitionLength - void testShuffleWrite(VeloxShuffleWriter& shuffleWriter, std::vector vectors) { - for (auto& vector : vectors) { - splitRowVector(shuffleWriter, vector); - } - ASSERT_NOT_OK(shuffleWriter.stop()); - // verify data file - checkFileExists(shuffleWriter.dataFile()); - // verify output temporary files - const auto& lengths = shuffleWriter.partitionLengths(); - ASSERT_EQ(lengths.size(), 1); - - auto schema = getArrowSchema(vectors[0]); - std::vector deserializedVectors; - setReadableFile(shuffleWriter.dataFile()); - getRowVectors(schema, deserializedVectors); - - ASSERT_EQ(deserializedVectors.size(), vectors.size()); - for (int32_t i = 0; i < deserializedVectors.size(); i++) { - velox::test::assertEqualVectors(vectors[i], deserializedVectors[i]); - } + // Split > 1 RowVector. + { + auto shuffleWriter = createShuffleWriter(); + testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector2_, inputVector1_}); } - - void shuffleWriteReadMultiBlocks( - VeloxShuffleWriter& shuffleWriter, - int32_t expectPartitionLength, - TypePtr dataType, - std::vector> expectedVectors) { /* blockId = pid, rowVector in block */ - ASSERT_NOT_OK(shuffleWriter.stop()); - // verify data file - checkFileExists(shuffleWriter.dataFile()); - // verify output temporary files - const auto& lengths = shuffleWriter.partitionLengths(); - ASSERT_EQ(lengths.size(), expectPartitionLength); - int64_t lengthSum = std::accumulate(lengths.begin(), lengths.end(), 0); - auto schema = toArrowSchema(dataType, pool()); - setReadableFile(shuffleWriter.dataFile()); - ASSERT_EQ(*file_->GetSize(), lengthSum); - for (int32_t i = 0; i < expectPartitionLength; i++) { - if (expectedVectors[i].size() == 0) { - ASSERT_EQ(lengths[i], 0); - } else { - std::vector deserializedVectors; - getRowVectors(schema, deserializedVectors); - if (i != 0) { - ASSERT_NOT_OK(file_->Advance(lengths[i - 1])); - } - ASSERT_EQ(expectedVectors[i].size(), deserializedVectors.size()); - for (int32_t j = 0; j < expectedVectors[i].size(); j++) { - velox::test::assertEqualVectors(expectedVectors[i][j], deserializedVectors[j]); - } - } - } + // Test not compress small buffer. + { + shuffleWriterOptions_.compression_type = arrow::Compression::LZ4_FRAME; + shuffleWriterOptions_.compression_threshold = 1024; + auto shuffleWriter = createShuffleWriter(); + testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector1_}); } - - void testShuffleWriteMultiBlocks( - VeloxShuffleWriter& shuffleWriter, - std::vector> batches, - int32_t expectPartitionLength, - TypePtr dataType, - std::vector> expectedVectors) { /* blockId = pid, rowVector in block */ - for (auto& batch : batches) { - GLUTEN_THROW_NOT_OK(shuffleWriter.split(batch, ShuffleWriter::kMinMemLimit)); - } - shuffleWriteReadMultiBlocks(shuffleWriter, expectPartitionLength, dataType, expectedVectors); + // Split null RowVector. + { + auto shuffleWriter = createShuffleWriter(); + auto vector = makeRowVector({ + makeNullableFlatVector({std::nullopt}), + makeNullableFlatVector({std::nullopt}), + }); + testShuffleWrite(*shuffleWriter, {vector}); } - - void testShuffleWriteMultiBlocks( - VeloxShuffleWriter& shuffleWriter, - std::vector vectors, - int32_t expectPartitionLength, - TypePtr dataType, - std::vector> expectedVectors) { - for (auto& vector : vectors) { - splitRowVector(shuffleWriter, vector); - } - shuffleWriteReadMultiBlocks(shuffleWriter, expectPartitionLength, dataType, expectedVectors); + // Other types. + { + auto shuffleWriter = createShuffleWriter(); + auto vector = makeRowVector({ + makeNullableFlatVector({std::nullopt, 1}), + makeNullableFlatVector({std::nullopt, "10"}), + makeNullableFlatVector({232, 34567235}, DECIMAL(12, 4)), + makeNullableFlatVector({232, 34567235}, DECIMAL(20, 4)), + makeFlatVector( + 2, [](vector_size_t row) { return row % 2; }, nullEvery(5), DATE()), + makeNullableFlatVector({std::nullopt, 1}), + makeRowVector({ + makeFlatVector({1, 3}), + makeNullableFlatVector({std::nullopt, "de"}), + }), + makeNullableFlatVector({std::nullopt, "10 I'm not inline string"}), + makeArrayVector({ + {1, 2, 3, 4, 5}, + {1, 2, 3}, + }), + makeMapVector({{{1, "str1000"}, {2, "str2000"}}, {{3, "str3000"}, {4, "str4000"}}}), + }); + testShuffleWrite(*shuffleWriter, {vector}); } - - int64_t splitRowVectorAndSpill(std::vector vectors, bool shrink) { - for (auto vector : vectors) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, vector)); - } - - auto targetEvicted = shuffleWriter_->cachedPayloadSize(); - if (shrink) { - targetEvicted += shuffleWriter_->partitionBufferSize(); - } - int64_t evicted; - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(targetEvicted, &evicted)); - - return evicted; - }; - - std::shared_ptr tmpDir1_; - std::shared_ptr tmpDir2_; - - ShuffleWriterOptions shuffleWriterOptions_; - std::shared_ptr shuffleWriter_; - - std::shared_ptr partitionWriterCreator_; - - std::shared_ptr file_; - - velox::RowVectorPtr inputVector1_; - velox::RowVectorPtr inputVector2_; - velox::RowVectorPtr hashInputVector1_; - velox::RowVectorPtr hashInputVector2_; - std::vector hashPartitionIds_{1, 2}; - - std::shared_ptr arrowPool_ = defaultArrowMemoryPool(); -}; - -TEST_P(VeloxShuffleWriterTest, singlePart1Vector) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - testShuffleWrite(*shuffleWriter, {inputVector1_}); -} - -TEST_P(VeloxShuffleWriterTest, singlePart3Vectors) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector2_, inputVector1_}); } -TEST_P(VeloxShuffleWriterTest, singlePartCompressSmallBuffer) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - shuffleWriterOptions_.compression_type = arrow::Compression::LZ4_FRAME; - shuffleWriterOptions_.buffer_compress_threshold = 1024; - - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - testShuffleWrite(*shuffleWriter, {inputVector1_, inputVector1_}); -} - -TEST_P(VeloxShuffleWriterTest, singlePartNullVector) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - auto vector = makeRowVector({ - makeNullableFlatVector({std::nullopt}), - makeNullableFlatVector({std::nullopt}), - }); - testShuffleWrite(*shuffleWriter, {vector}); -} - -TEST_P(VeloxShuffleWriterTest, singlePartOtherType) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - auto vector = makeRowVector({ - makeNullableFlatVector({std::nullopt, 1}), - makeNullableFlatVector({std::nullopt, "10"}), - makeShortDecimalFlatVector({232, 34567235}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({232, 34567235}, DECIMAL(20, 4)), - makeFlatVector( - 2, [](vector_size_t row) { return Date{row % 2}; }, nullEvery(5)), - }); - testShuffleWrite(*shuffleWriter, {vector}); -} - -TEST_P(VeloxShuffleWriterTest, singlePartComplexType) { - shuffleWriterOptions_.buffer_size = 10; - shuffleWriterOptions_.partitioning_name = "single"; - - GLUTEN_ASSIGN_OR_THROW( - auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - auto vector = makeRowVector({ - makeNullableFlatVector({std::nullopt, 1}), - makeRowVector({ - makeFlatVector({1, 3}), - makeNullableFlatVector({std::nullopt, "de"}), - }), - makeNullableFlatVector({std::nullopt, "10 I'm not inline string"}), - makeArrayVector({ - {1, 2, 3, 4, 5}, - {1, 2, 3}, - }), - makeMapVector({{{1, "str1000"}, {2, "str2000"}}, {{3, "str3000"}, {4, "str4000"}}}), - }); - testShuffleWrite(*shuffleWriter, {vector}); -} - -TEST_P(VeloxShuffleWriterTest, hashPart1Vector) { - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning_name = "hash"; - - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) +TEST_P(HashPartitioningShuffleWriter, hashPart1Vector) { + auto shuffleWriter = createShuffleWriter(); auto vector = makeRowVector({ makeFlatVector({1, 2, 1, 2}), makeNullableFlatVector({1, 2, 3, std::nullopt}), makeFlatVector({1, 2, 3, 4}), makeFlatVector({"nn", "re", "fr", "juiu"}), - makeShortDecimalFlatVector({232, 34567235, 1212, 4567}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({232, 34567235, 1212, 4567}, DECIMAL(20, 4)), - makeFlatVector( - 4, [](vector_size_t row) { return Date{row % 2}; }, nullEvery(5)), + makeFlatVector({232, 34567235, 1212, 4567}, DECIMAL(12, 4)), + makeFlatVector({232, 34567235, 1212, 4567}, DECIMAL(20, 4)), + makeFlatVector( + 4, [](vector_size_t row) { return row % 2; }, nullEvery(5), DATE()), makeFlatVector( 4, [](vector_size_t row) { @@ -466,10 +145,10 @@ TEST_P(VeloxShuffleWriterTest, hashPart1Vector) { makeNullableFlatVector({1, 2, 3, std::nullopt}), makeFlatVector({1, 2, 3, 4}), makeFlatVector({"nn", "re", "fr", "juiu"}), - makeShortDecimalFlatVector({232, 34567235, 1212, 4567}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({232, 34567235, 1212, 4567}, DECIMAL(20, 4)), - makeFlatVector( - 4, [](vector_size_t row) { return Date{row % 2}; }, nullEvery(5)), + makeFlatVector({232, 34567235, 1212, 4567}, DECIMAL(12, 4)), + makeFlatVector({232, 34567235, 1212, 4567}, DECIMAL(20, 4)), + makeFlatVector( + 4, [](vector_size_t row) { return row % 2; }, nullEvery(5), DATE()), makeFlatVector( 4, [](vector_size_t row) { @@ -482,9 +161,9 @@ TEST_P(VeloxShuffleWriterTest, hashPart1Vector) { makeNullableFlatVector({2, std::nullopt}), makeFlatVector({2, 4}), makeFlatVector({"re", "juiu"}), - makeShortDecimalFlatVector({34567235, 4567}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({34567235, 4567}, DECIMAL(20, 4)), - makeFlatVector({Date(1), Date(1)}), + makeFlatVector({34567235, 4567}, DECIMAL(12, 4)), + makeFlatVector({34567235, 4567}, DECIMAL(20, 4)), + makeFlatVector({1, 1}, DATE()), makeFlatVector({Timestamp(1, 0), Timestamp(1, 0)}), }); @@ -492,21 +171,17 @@ TEST_P(VeloxShuffleWriterTest, hashPart1Vector) { makeNullableFlatVector({1, 3}), makeFlatVector({1, 3}), makeFlatVector({"nn", "fr"}), - makeShortDecimalFlatVector({232, 1212}, DECIMAL(12, 4)), - makeLongDecimalFlatVector({232, 1212}, DECIMAL(20, 4)), - makeNullableFlatVector({std::nullopt, Date(0)}), + makeFlatVector({232, 1212}, DECIMAL(12, 4)), + makeFlatVector({232, 1212}, DECIMAL(20, 4)), + makeNullableFlatVector({std::nullopt, 0}, DATE()), makeNullableFlatVector({std::nullopt, Timestamp(0, 0)}), }); - testShuffleWriteMultiBlocks(*shuffleWriter_, {vector}, 2, dataVector->type(), {{firstBlock}, {secondBlock}}); + testShuffleWriteMultiBlocks(*shuffleWriter, {vector}, 2, dataVector->type(), {{firstBlock}, {secondBlock}}); } -TEST_P(VeloxShuffleWriterTest, hashPart1VectorComplexType) { - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning_name = "hash"; - - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) +TEST_P(HashPartitioningShuffleWriter, hashPart1VectorComplexType) { + auto shuffleWriter = createShuffleWriter(); std::vector children = { makeNullableFlatVector({std::nullopt, 1}), makeRowVector({ @@ -550,15 +225,11 @@ TEST_P(VeloxShuffleWriterTest, hashPart1VectorComplexType) { makeMapVector({{{1, "str1000"}, {2, "str2000"}}}), }); - testShuffleWriteMultiBlocks(*shuffleWriter_, {vector}, 2, dataVector->type(), {{firstBlock}, {secondBlock}}); + testShuffleWriteMultiBlocks(*shuffleWriter, {vector}, 2, dataVector->type(), {{firstBlock}, {secondBlock}}); } -TEST_P(VeloxShuffleWriterTest, hashPart3Vectors) { - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning_name = "hash"; - - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) +TEST_P(HashPartitioningShuffleWriter, hashPart3Vectors) { + auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 5, 6, 7, 9}); auto block2Pid1 = takeRows(inputVector2_, {}); @@ -567,19 +238,15 @@ TEST_P(VeloxShuffleWriterTest, hashPart3Vectors) { auto block2Pid2 = takeRows(inputVector2_, {0, 1}); testShuffleWriteMultiBlocks( - *shuffleWriter_, + *shuffleWriter, {hashInputVector1_, hashInputVector2_, hashInputVector1_}, 2, inputVector1_->type(), {{block1Pid2, block2Pid2, block1Pid2}, {block1Pid1, block1Pid1}}); } -TEST_P(VeloxShuffleWriterTest, roundRobin) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); +TEST_P(RangePartitioningShuffleWriter, rangePartition) { + auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 2, 4, 6, 8}); auto block2Pid1 = takeRows(inputVector2_, {0}); @@ -588,30 +255,15 @@ TEST_P(VeloxShuffleWriterTest, roundRobin) { auto block2Pid2 = takeRows(inputVector2_, {1}); testShuffleWriteMultiBlocks( - *shuffleWriter_, - {inputVector1_, inputVector2_, inputVector1_}, + *shuffleWriter, + {compositeBatch1_, compositeBatch2_, compositeBatch1_}, 2, inputVector1_->type(), {{block1Pid1, block2Pid1, block1Pid1}, {block1Pid2, block2Pid2, block1Pid2}}); } -TEST_P(VeloxShuffleWriterTest, rangePartition) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.partitioning_name = "range"; - - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)) - - auto pid1 = makeRowVector({makeFlatVector({0, 1, 0, 1, 0, 1, 0, 1, 0, 1})}); - auto rangeVector1 = makeRowVector(inputVector1_->children()); - auto compositeBatch1 = CompositeColumnarBatch::create( - {std::make_shared(pid1), std::make_shared(rangeVector1)}); - - auto pid2 = makeRowVector({makeFlatVector({0, 1})}); - auto rangeVector2 = makeRowVector(inputVector2_->children()); - auto compositeBatch2 = CompositeColumnarBatch::create( - {std::make_shared(pid2), std::make_shared(rangeVector2)}); +TEST_P(RoundRobinPartitioningShuffleWriter, roundRobin) { + auto shuffleWriter = createShuffleWriter(); auto block1Pid1 = takeRows(inputVector1_, {0, 2, 4, 6, 8}); auto block2Pid1 = takeRows(inputVector2_, {0}); @@ -620,413 +272,387 @@ TEST_P(VeloxShuffleWriterTest, rangePartition) { auto block2Pid2 = takeRows(inputVector2_, {1}); testShuffleWriteMultiBlocks( - *shuffleWriter_, - {compositeBatch1, compositeBatch2, compositeBatch1}, + *shuffleWriter, + {inputVector1_, inputVector2_, inputVector1_}, 2, inputVector1_->type(), {{block1Pid1, block2Pid1, block1Pid1}, {block1Pid2, block2Pid2, block1Pid2}}); } -TEST_P(VeloxShuffleWriterTest, memoryLeak) { - std::shared_ptr pool = std::make_shared(); +TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceRealloc) { + shuffleWriterOptions_.buffer_realloc_threshold = 0; // Force re-alloc on buffer size changed. + auto shuffleWriter = createShuffleWriter(); + + // First spilt no null. + auto inputNoNull = inputVectorNoNull_; - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.memory_pool = pool.get(); - shuffleWriterOptions_.partitioning_name = "rr"; + // Second split has null. Continue filling current partition buffers. + std::vector intHasNull = { + makeNullableFlatVector({std::nullopt, 1}), + makeNullableFlatVector({std::nullopt, -1}), + makeNullableFlatVector({std::nullopt, 100}), + makeNullableFlatVector({0, 1}), + makeNullableFlatVector({0, 0.142857}), + makeNullableFlatVector({false, true}), + makeNullableFlatVector({"", "alice"}), + makeNullableFlatVector({"alice", ""}), + }; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); + auto inputHasNull = makeRowVector(intHasNull); + // Split first input no null. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputNoNull)); + // Split second input, continue filling but update null. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputHasNull)); - splitRowVector(*shuffleWriter_, inputVector1_); - splitRowVector(*shuffleWriter_, inputVector2_); - splitRowVector(*shuffleWriter_, inputVector1_); + // Split first input again. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputNoNull)); + // Check when buffer is full, evict current buffers and reuse. + auto cachedPayloadSize = shuffleWriter->cachedPayloadSize(); + auto partitionBufferBeforeEvict = shuffleWriter->partitionBufferSize(); + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(cachedPayloadSize, &evicted)); + // Check only cached data being spilled. + ASSERT_EQ(evicted, cachedPayloadSize); + VELOX_CHECK_EQ(shuffleWriter->partitionBufferSize(), partitionBufferBeforeEvict); - ASSERT_NOT_OK(shuffleWriter_->stop()); + // Split more data with null. New buffer size is larger. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - ASSERT_TRUE(pool->bytes_allocated() == 0); - shuffleWriter_.reset(); - ASSERT_TRUE(pool->bytes_allocated() == 0); + // Split more data with null. New buffer size is smaller. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + + // Split more data with null. New buffer size is larger and current data is preserved. + // Evict cached data first. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->cachedPayloadSize(), &evicted)); + // Set a large buffer size. + shuffleWriter->options().buffer_size = 100; + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + // No data got evicted so the cached size is 0. + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + + ASSERT_NOT_OK(shuffleWriter->stop()); } -TEST_P(VeloxShuffleWriterTest, spillFailWithOutOfMemory) { - auto pool = std::make_shared(0); +TEST_P(RoundRobinPartitioningShuffleWriter, preAllocForceReuse) { + shuffleWriterOptions_.buffer_realloc_threshold = 100; // Force re-alloc on buffer size changed. + auto shuffleWriter = createShuffleWriter(); - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 4; - shuffleWriterOptions_.memory_pool = pool.get(); - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); + // First spilt no null. + auto inputNoNull = inputVectorNoNull_; + // Second split has null int, null string and non-null string, + auto inputFixedWidthHasNull = inputVector1_; + // Third split has null string. + std::vector stringHasNull = { + makeNullableFlatVector({0, 1}), + makeNullableFlatVector({0, -1}), + makeNullableFlatVector({0, 100}), + makeNullableFlatVector({0, 1}), + makeNullableFlatVector({0, 0.142857}), + makeNullableFlatVector({false, true}), + makeNullableFlatVector({std::nullopt, std::nullopt}), + makeNullableFlatVector({std::nullopt, std::nullopt}), + }; + auto inputStringHasNull = makeRowVector(stringHasNull); - auto status = splitRowVectorStatus(*shuffleWriter_, inputVector1_); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputNoNull)); + // Split more data with null. Already filled + to be filled > buffer size, Buffer is resized larger. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputFixedWidthHasNull)); + // Split more data with null. Already filled + to be filled > buffer size, newSize is smaller so buffer is not + // resized. + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputStringHasNull)); - // should return OOM status because there's no partition buffer to spill - ASSERT_TRUE(status.IsOutOfMemory()); - ASSERT_NOT_OK(shuffleWriter_->stop()); + ASSERT_NOT_OK(shuffleWriter->stop()); } -TEST_P(VeloxShuffleWriterTest, TestSpillLargestPartition) { - std::shared_ptr pool = std::make_shared(9 * 1024 * 1024); - // pool = std::make_shared(pool.get()); +TEST_P(RoundRobinPartitioningShuffleWriter, spillVerifyResult) { + auto shuffleWriter = createShuffleWriter(); - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 4; - // shuffleWriterOptions_.memory_pool = pool.get(); - shuffleWriterOptions_.compression_type = arrow::Compression::UNCOMPRESSED; - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); - - for (int i = 0; i < 100; ++i) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + + // Clear buffers and evict payloads and cache. + for (auto pid : {0, 1}) { + GLUTEN_ASSIGN_OR_THROW(auto payload, shuffleWriter->createPayloadFromBuffer(pid, true)); + if (payload) { + ASSERT_NOT_OK(shuffleWriter->evictPayload(pid, std::move(payload))); + } } - ASSERT_NOT_OK(shuffleWriter_->stop()); -} -TEST_P(VeloxShuffleWriterTest, TestSplitSpillAndShrink) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 100; // Set a large buffer size to make sure there are spaces to shrink. - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - for (int i = 0; i < 100; ++i) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); + // Evict all payloads and spill. + int64_t evicted; + auto cachedPayloadSize = shuffleWriter->cachedPayloadSize(); + auto partitionBufferSize = shuffleWriter->partitionBufferSize(); + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(cachedPayloadSize + partitionBufferSize, &evicted)); - auto bufferSize = shuffleWriter_->partitionBufferSize(); - auto payloadSize = shuffleWriter_->cachedPayloadSize(); - int64_t evicted; - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(payloadSize + bufferSize, &evicted)); - ASSERT_GT(evicted, 0); - } + ASSERT_EQ(evicted, cachedPayloadSize + partitionBufferSize); + + // No more cached payloads after spill. + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + ASSERT_EQ(shuffleWriter->partitionBufferSize(), 0); + + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + + auto block1Pid1 = takeRows(inputVector1_, {0, 2, 4, 6, 8}); + auto block1Pid2 = takeRows(inputVector1_, {1, 3, 5, 7, 9}); - ASSERT_NOT_OK(shuffleWriter_->stop()); + // Stop and verify. + shuffleWriteReadMultiBlocks( + *shuffleWriter, + 2, + inputVector1_->type(), + {{block1Pid1, block1Pid1, block1Pid1}, {block1Pid2, block1Pid2, block1Pid2}}); } -TEST_P(VeloxShuffleWriterTest, TestStopShrinkAndSpill) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 100; // Set a large buffer size to make sure there are spaces to shrink. - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); - - for (int i = 0; i < 100; ++i) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - } +TEST_F(VeloxShuffleWriterMemoryTest, memoryLeak) { + std::shared_ptr pool = std::make_shared(); + shuffleWriterOptions_.memory_pool = pool.get(); + shuffleWriterOptions_.buffer_size = 4; - auto bufferSize = shuffleWriter_->partitionBufferSize(); - auto payloadSize = shuffleWriter_->cachedPayloadSize(); - if (shuffleWriterOptions_.partition_writer_type == PartitionWriterType::kLocal) { - // No evict triggered, the cached payload should not be empty. - ASSERT_GT(payloadSize, 0); - } + auto shuffleWriter = createShuffleWriter(); - int64_t evicted; - shuffleWriter_->setSplitState(SplitState::kStop); - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(payloadSize + bufferSize, &evicted)); - // Total evicted should be greater than payloadSize, to test shrinking has been triggered. - ASSERT_GT(evicted, payloadSize); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - ASSERT_NOT_OK(shuffleWriter_->stop()); -} + ASSERT_NOT_OK(shuffleWriter->stop()); -TEST_P(VeloxShuffleWriterTest, TestSpillOnStop) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 100; // Set a large buffer size to make sure there are spaces to shrink. - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); - - auto evicted = splitRowVectorAndSpill({inputVector1_, inputVector2_, inputVector1_}, true); - ASSERT_GT(evicted, 0); - ASSERT_EQ(shuffleWriter_->cachedPayloadSize(), 0); - - // Split multiple times, to get non-empty partition buffers and cached payloads. - for (int i = 0; i < 100; ++i) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - } + ASSERT_TRUE(pool->bytes_allocated() == 0); + shuffleWriter.reset(); + ASSERT_TRUE(pool->bytes_allocated() == 0); +} - if (shuffleWriterOptions_.partition_writer_type == PartitionWriterType::kLocal) { - // No evict triggered, the cached payload should not be empty. - ASSERT_GT(shuffleWriter_->cachedPayloadSize(), 0); - } +TEST_F(VeloxShuffleWriterMemoryTest, spillFailWithOutOfMemory) { + std::shared_ptr pool = std::make_shared(0); + shuffleWriterOptions_.memory_pool = pool.get(); + shuffleWriterOptions_.buffer_size = 4; - // Spill on stop. - shuffleWriter_->setSplitState(SplitState::kStop); + auto shuffleWriter = createShuffleWriter(); - auto payloadSize = shuffleWriter_->cachedPayloadSize(); - auto bufferSize = shuffleWriter_->partitionBufferSize(); - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(payloadSize + bufferSize, &evicted)); - // Total evicted should be greater than payloadSize, to test shrinking has been triggered. - ASSERT_GT(evicted, payloadSize); + auto status = splitRowVector(*shuffleWriter, inputVector1_); - ASSERT_NOT_OK(shuffleWriter_->stop()); + // Should return OOM status because there's no partition buffer to spill. + ASSERT_TRUE(status.IsOutOfMemory()); } -TEST_P(VeloxShuffleWriterTest, TestSpill) { - int32_t numPartitions = 4; - shuffleWriterOptions_.buffer_size = 1; // Set a small buffer size to force clear and cache buffers for each split. - shuffleWriterOptions_.partitioning_name = "hash"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); +TEST_F(VeloxShuffleWriterMemoryTest, kInit) { + shuffleWriterOptions_.buffer_size = 4; + auto shuffleWriter = createShuffleWriter(); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, hashInputVector1_)); + // Test spill all partition buffers. + { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - // Clear buffers and evict payloads. - for (auto pid : hashPartitionIds_) { - std::unique_ptr payload; - ARROW_ASSIGN_OR_THROW(payload, shuffleWriter_->createPayloadFromBuffer(pid, true)); - if (payload) { - ASSERT_NOT_OK(shuffleWriter_->evictPayload(pid, std::move(payload))); - } + auto bufferSize = shuffleWriter->partitionBufferSize(); + auto payloadSize = shuffleWriter->cachedPayloadSize(); + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(payloadSize + bufferSize, &evicted)); + ASSERT_EQ(evicted, payloadSize + bufferSize); + // No cached payload after evict. + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + // All partition buffers should be evicted. + ASSERT_EQ(shuffleWriter->partitionBufferSize(), 0); } - // Evict all payloads. - auto evicted = splitRowVectorAndSpill({hashInputVector1_}, true); - if (shuffleWriterOptions_.partition_writer_type == PartitionWriterType::kLocal) { - ASSERT_GT(evicted, 0); + // Test spill minimum-size partition buffers. + { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + + auto bufferSize = shuffleWriter->partitionBufferSize(); + auto payloadSize = shuffleWriter->cachedPayloadSize(); + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(payloadSize + 1, &evicted)); + ASSERT_GT(evicted, payloadSize); + // No cached payload after evict. + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + ASSERT_LE(shuffleWriter->partitionBufferSize(), bufferSize); + // Not all partition buffers was evicted. + ASSERT_GT(shuffleWriter->partitionBufferSize(), 0); } - // No more cached payloads after spill. - ASSERT_EQ(shuffleWriter_->cachedPayloadSize(), 0); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, hashInputVector1_)); + // Test spill empty partition buffers. + { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + + // Clear buffers then the size after shrink will be 0. + for (auto pid = 0; pid < kDefaultShufflePartitions; ++pid) { + GLUTEN_ASSIGN_OR_THROW(auto payload, shuffleWriter->createPayloadFromBuffer(pid, true)); + if (payload) { + ASSERT_NOT_OK(shuffleWriter->evictPayload(pid, std::move(payload))); + } + } - auto block1Pid1 = takeRows(inputVector1_, {0, 5, 6, 7, 9}); - auto block1Pid2 = takeRows(inputVector1_, {1, 2, 3, 4, 8}); + auto bufferSize = shuffleWriter->partitionBufferSize(); + auto payloadSize = shuffleWriter->cachedPayloadSize(); + int64_t evicted; + // Evict payload and shrink min-size buffer. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(payloadSize + 1, &evicted)); + ASSERT_GT(evicted, payloadSize); + ASSERT_GT(shuffleWriter->partitionBufferSize(), 0); + // Evict empty partition buffers. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(bufferSize, &evicted)); + ASSERT_GT(evicted, 0); + ASSERT_EQ(shuffleWriter->partitionBufferSize(), 0); + // Evict again. No reclaimable space. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(1, &evicted)); + ASSERT_EQ(evicted, 0); + } - shuffleWriteReadMultiBlocks( - *shuffleWriter_, - numPartitions, - inputVector1_->type(), - {{}, {block1Pid1, block1Pid1, block1Pid1}, {block1Pid2, block1Pid2, block1Pid2}, {}}); + ASSERT_NOT_OK(shuffleWriter->stop()); } -TEST_P(VeloxShuffleWriterTest, TestShrinkZeroSizeBuffer) { - // Test 2 cases: - // 1. partition buffer size before shrink is 0. - // 2. partition buffer size after shrink is 0. - int32_t numPartitions = 200; // Set a large number of partitions to create empty partition buffers. - shuffleWriterOptions_.buffer_size = 100; // Set a large buffer size to make sure there are spaces to shrink. - shuffleWriterOptions_.partitioning_name = "hash"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); - auto evicted = splitRowVectorAndSpill({hashInputVector1_, hashInputVector2_, hashInputVector1_}, true); - ASSERT_GT(evicted, 0); - - // Clear buffers then the size after shrink will be 0. - for (auto pid : hashPartitionIds_) { - std::unique_ptr payload; - ARROW_ASSIGN_OR_THROW(payload, shuffleWriter_->createPayloadFromBuffer(pid, true)); - if (payload) { - ASSERT_NOT_OK(shuffleWriter_->evictPayload(pid, std::move(payload))); - } - } +TEST_F(VeloxShuffleWriterMemoryTest, kInitSingle) { + shuffleWriterOptions_.partitioning = Partitioning::kSingle; + shuffleWriterOptions_.buffer_size = 4; + auto shuffleWriter = createShuffleWriter(); + + { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - auto bufferSize = shuffleWriter_->partitionBufferSize(); - auto payloadSize = shuffleWriter_->cachedPayloadSize(); - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(payloadSize + bufferSize, &evicted)); - // All cached payloads and partition buffer memory should be evicted. - ASSERT_EQ(evicted, payloadSize + bufferSize); - // All buffers should be released. - ASSERT_EQ(shuffleWriter_->partitionBufferSize(), 0); + auto payloadSize = shuffleWriter->cachedPayloadSize(); + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(payloadSize + 1, &evicted)); + ASSERT_EQ(evicted, payloadSize); + // No cached payload after evict. + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + } - ASSERT_NOT_OK(shuffleWriter_->stop()); + ASSERT_NOT_OK(shuffleWriter->stop()); } -TEST_P(VeloxShuffleWriterTest, SmallBufferSizeNoShrink) { - int32_t numPartitions = 4; // Set a large number of partitions to create empty partition buffers. - shuffleWriterOptions_.buffer_size = 1; // Set a small buffer size to test no space to shrink. - shuffleWriterOptions_.partitioning_name = "hash"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); +TEST_F(VeloxShuffleWriterMemoryTest, kSplit) { + shuffleWriterOptions_.buffer_size = 4; + auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_.memory_pool); + shuffleWriterOptions_.memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, hashInputVector1_)); + pool.setEvictable(shuffleWriter.get()); - int64_t evicted = 0; - auto bufferSize = shuffleWriter_->partitionBufferSize(); - auto payloadSize = shuffleWriter_->cachedPayloadSize(); - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(payloadSize + bufferSize, &evicted)); - ASSERT_EQ(evicted, 0); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - ASSERT_NOT_OK(shuffleWriter_->stop()); -} + // Trigger spill for the next split. + ASSERT_TRUE(pool.checkEvict(pool.bytes_allocated(), [&] { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + })); -TEST_P(VeloxShuffleWriterTest, SinglePartitioningNoShrink) { - shuffleWriterOptions_.partitioning_name = "single"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)); + ASSERT_NOT_OK(shuffleWriter->stop()); +} - // Split multiple times, to get non-empty partition buffers and cached payloads. - for (int i = 0; i < 100; ++i) { - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - } +TEST_F(VeloxShuffleWriterMemoryTest, kSplitSingle) { + shuffleWriterOptions_.partitioning = Partitioning::kSingle; + auto pool = SelfEvictedMemoryPool(shuffleWriterOptions_.memory_pool); + shuffleWriterOptions_.memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); - ASSERT_EQ(shuffleWriter_->getSplitState(), SplitState::kInit); + pool.setEvictable(shuffleWriter.get()); - // No partition buffers for single partitioner. - ASSERT_EQ(shuffleWriter_->partitionBufferSize(), 0); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - int64_t evicted = 0; - auto cachedPayloadSize = shuffleWriter_->cachedPayloadSize(); - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(cachedPayloadSize + 1, &evicted)); - // No shrink. - ASSERT_EQ(evicted, cachedPayloadSize); - // No more cached payloads after spill. - ASSERT_EQ(shuffleWriter_->cachedPayloadSize(), 0); + // Trigger spill for the next split. + ASSERT_TRUE(pool.checkEvict( + shuffleWriter->cachedPayloadSize() * 2, [&] { ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); })); - // No more space to spill or shrink. - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(1, &evicted)); - ASSERT_EQ(evicted, 0); - ASSERT_NOT_OK(shuffleWriter_->stop()); + ASSERT_NOT_OK(shuffleWriter->stop()); } -TEST_P(VeloxShuffleWriterTest, PreAllocPartitionBuffer1) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 2; // Set a small buffer size. - shuffleWriterOptions_.buffer_realloc_threshold = 0; // Force re-alloc on buffer size changed. - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); +TEST_F(VeloxShuffleWriterMemoryTest, kStop) { + auto delegated = shuffleWriterOptions_.memory_pool; + for (const auto partitioning : {Partitioning::kSingle, Partitioning::kRoundRobin}) { + shuffleWriterOptions_.partitioning = partitioning; + shuffleWriterOptions_.buffer_size = 4; + auto pool = SelfEvictedMemoryPool(delegated); + shuffleWriterOptions_.memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); + + pool.setEvictable(shuffleWriter.get()); + + for (int i = 0; i < 10; ++i) { + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector2_)); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + } - // First spilt no null. - std::vector noNull = { - makeFlatVector({0, 1}), - makeFlatVector({0, -1}), - makeFlatVector({0, 100}), - makeFlatVector({0, 1}), - makeFlatVector({0, 0.142857}), - makeFlatVector({false, true}), - makeFlatVector({"", "alice"}), - makeFlatVector({"alice", ""}), - }; - auto inputNoNull = makeRowVector(noNull); + // Trigger spill during stop. + // For single partitioning, spill is triggered by allocating buffered output stream. + ASSERT_TRUE(pool.checkEvict(pool.bytes_allocated(), [&] { ASSERT_NOT_OK(shuffleWriter->stop()); })); + } +} - // Second split has null. Continue filling current partition buffers. - std::vector intHasNull = { - makeNullableFlatVector({std::nullopt, 1}), - makeNullableFlatVector({std::nullopt, -1}), - makeNullableFlatVector({std::nullopt, 100}), - makeNullableFlatVector({0, 1}), - makeNullableFlatVector({0, 0.142857}), - makeNullableFlatVector({false, true}), - makeNullableFlatVector({"", "alice"}), - makeNullableFlatVector({"alice", ""}), - }; +TEST_F(VeloxShuffleWriterMemoryTest, kUnevictable) { + auto delegated = shuffleWriterOptions_.memory_pool; + shuffleWriterOptions_.buffer_size = 4; + auto pool = SelfEvictedMemoryPool(delegated); + shuffleWriterOptions_.memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); - auto inputHasNull = makeRowVector(intHasNull); - // Split first input no null. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputNoNull)); - // Split second input, continue filling but update null. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputHasNull)); + pool.setEvictable(shuffleWriter.get()); - // Split first input again. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputNoNull)); - // Check when buffer is full, evict current buffers and reuse. - auto cachedPayloadSize = shuffleWriter_->cachedPayloadSize(); - auto partitionBufferBeforeEvict = shuffleWriter_->partitionBufferSize(); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); + + // First evict cached payloads. int64_t evicted; - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(cachedPayloadSize, &evicted)); - // Check only cached data being spilled. - ASSERT_EQ(evicted, cachedPayloadSize); - ARROW_CHECK_EQ(shuffleWriter_->partitionBufferSize(), partitionBufferBeforeEvict); + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->cachedPayloadSize(), &evicted)); + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + ASSERT_GT(shuffleWriter->partitionBufferSize(), 0); + // Set limited capacity. + pool.setCapacity(0); + // Evict again. Because no cached payload to evict, it will try to compress and evict partition buffers. + // Throws OOM during allocating compression buffers. + auto status = shuffleWriter->evictFixedSize(shuffleWriter->partitionBufferSize(), &evicted); + ASSERT_TRUE(status.IsOutOfMemory()); +} - // Split more data with null. New buffer size is larger. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); +TEST_F(VeloxShuffleWriterMemoryTest, kUnevictableSingle) { + auto delegated = shuffleWriterOptions_.memory_pool; + shuffleWriterOptions_.partitioning = Partitioning::kSingle; + auto pool = SelfEvictedMemoryPool(delegated); + shuffleWriterOptions_.memory_pool = &pool; + auto shuffleWriter = createShuffleWriter(); - // Split more data with null. New buffer size is smaller. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector2_)); + pool.setEvictable(shuffleWriter.get()); - // Split more data with null. New buffer size is larger and current data is preserved. - // Evict cached data first. - ASSERT_NOT_OK(shuffleWriter_->evictFixedSize(shuffleWriter_->cachedPayloadSize(), &evicted)); - // Set a large buffer size. - shuffleWriter_->options().buffer_size = 100; - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputVector1_)); - // No data got evicted so the cached size is 0. - ASSERT_EQ(shuffleWriter_->cachedPayloadSize(), 0); + ASSERT_NOT_OK(splitRowVector(*shuffleWriter, inputVector1_)); - ASSERT_NOT_OK(shuffleWriter_->stop()); + // First evict cached payloads. + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->cachedPayloadSize(), &evicted)); + ASSERT_EQ(shuffleWriter->cachedPayloadSize(), 0); + // Set limited capacity. + pool.setCapacity(0); + // Evict again. Single partitioning doesn't have partition buffers, so the evicted size is 0. + ASSERT_NOT_OK(shuffleWriter->evictFixedSize(shuffleWriter->partitionBufferSize(), &evicted)); + ASSERT_EQ(evicted, 0); } -TEST_P(VeloxShuffleWriterTest, PreAllocPartitionBuffer2) { - int32_t numPartitions = 2; - shuffleWriterOptions_.buffer_size = 2; // Set a small buffer size. - shuffleWriterOptions_.buffer_realloc_threshold = 100; // Set a large threshold to force buffer reused. - shuffleWriterOptions_.partitioning_name = "rr"; - ARROW_ASSIGN_OR_THROW( - shuffleWriter_, VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)); - - // First spilt no null. - std::vector noNull = { - makeFlatVector({0, 1}), - makeFlatVector({0, -1}), - makeFlatVector({0, 100}), - makeFlatVector({0, 1}), - makeFlatVector({0, 0.142857}), - makeFlatVector({false, true}), - makeFlatVector({"", "alice"}), - makeFlatVector({"alice", ""}), - }; - auto inputNoNull = makeRowVector(noNull); - - // Second split has null int. - std::vector fixedWithdHasNull = { - makeNullableFlatVector({0, 1, std::nullopt, std::nullopt}), - makeNullableFlatVector({0, -1, std::nullopt, std::nullopt}), - makeNullableFlatVector({0, 100, std::nullopt, std::nullopt}), - makeNullableFlatVector({0, 1, std::nullopt, std::nullopt}), - makeNullableFlatVector({0, 0.142857, std::nullopt, std::nullopt}), - makeNullableFlatVector({false, true, std::nullopt, std::nullopt}), - makeNullableFlatVector({"", "alice", "", ""}), - makeNullableFlatVector({"alice", "", "", ""}), - }; - auto inputFixedWidthHasNull = makeRowVector(fixedWithdHasNull); - - // Third split has null string. - std::vector stringHasNull = { - makeNullableFlatVector({0, 1}), - makeNullableFlatVector({0, -1}), - makeNullableFlatVector({0, 100}), - makeNullableFlatVector({0, 1}), - makeNullableFlatVector({0, 0.142857}), - makeNullableFlatVector({false, true}), - makeNullableFlatVector({std::nullopt, std::nullopt}), - makeNullableFlatVector({std::nullopt, std::nullopt}), - }; - auto inputStringHasNull = makeRowVector(stringHasNull); - - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputNoNull)); - // Split more data with null. Already filled + to be filled > buffer size, Buffer is resized larger. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputFixedWidthHasNull)); - // Split more data with null. Already filled + to be filled > buffer size, newSize is smaller so buffer is not - // resized. - ASSERT_NOT_OK(splitRowVectorStatus(*shuffleWriter_, inputStringHasNull)); +INSTANTIATE_TEST_SUITE_P( + VeloxShuffleWriteParam, + SinglePartitioningShuffleWriter, + ::testing::ValuesIn(kShuffleWriteTestParams)); - ASSERT_NOT_OK(shuffleWriter_->stop()); -} +INSTANTIATE_TEST_SUITE_P( + VeloxShuffleWriteParam, + RoundRobinPartitioningShuffleWriter, + ::testing::ValuesIn(kShuffleWriteTestParams)); INSTANTIATE_TEST_SUITE_P( VeloxShuffleWriteParam, - VeloxShuffleWriterTest, - ::testing::Values( - ShuffleTestParams{PartitionWriterType::kLocal, arrow::Compression::UNCOMPRESSED, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kLocal, arrow::Compression::LZ4_FRAME, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kLocal, arrow::Compression::ZSTD, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kCeleborn, arrow::Compression::UNCOMPRESSED, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kCeleborn, arrow::Compression::LZ4_FRAME, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kCeleborn, arrow::Compression::ZSTD, CompressionMode::BUFFER}, - ShuffleTestParams{PartitionWriterType::kLocal, arrow::Compression::UNCOMPRESSED, CompressionMode::ROWVECTOR}, - ShuffleTestParams{PartitionWriterType::kCeleborn, arrow::Compression::LZ4_FRAME, CompressionMode::ROWVECTOR})); + HashPartitioningShuffleWriter, + ::testing::ValuesIn(kShuffleWriteTestParams)); +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 a0d67985896a..0795f3927354 100644 --- a/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc +++ b/cpp/velox/tests/VeloxSubstraitRoundTripTest.cc @@ -76,25 +76,29 @@ class VeloxSubstraitRoundTripTest : public OperatorTestBase { assertQuery(samePlan, duckDbSql); } - void assertFailingPlanConversion( - const std::shared_ptr& plan, - const std::string& expectedErrorMessage) { - CursorParameters params; - params.planNode = plan; - VELOX_ASSERT_THROW(readCursor(params, [](auto /*task*/) {}), expectedErrorMessage); - - // Convert Velox Plan to Substrait Plan. - google::protobuf::Arena arena; - auto substraitPlan = veloxConvertor_->toSubstrait(arena, plan); - std::unordered_map sessionConf = {}; - std::shared_ptr substraitConverter_ = - std::make_shared(pool_.get(), sessionConf, true); - // Convert Substrait Plan to the same Velox Plan. - auto samePlan = substraitConverter_->toVeloxPlan(substraitPlan); - - // Assert velox again. - params.planNode = samePlan; - VELOX_ASSERT_THROW(readCursor(params, [](auto /*task*/) {}), expectedErrorMessage); + void assertFailingPlanConversion(const std::shared_ptr& plan, const std::string& errorMessage) { + try { + CursorParameters params; + params.planNode = plan; + readCursor(params, [](auto /*task*/) {}); + + // Convert Velox Plan to Substrait Plan. + google::protobuf::Arena arena; + auto substraitPlan = veloxConvertor_->toSubstrait(arena, plan); + std::unordered_map sessionConf = {}; + std::shared_ptr substraitConverter_ = + std::make_shared(pool_.get(), sessionConf, true); + // Convert Substrait Plan to the same Velox Plan. + auto samePlan = substraitConverter_->toVeloxPlan(substraitPlan); + + // Assert velox again. + params.planNode = samePlan; + readCursor(params, [](auto /*task*/) {}); + FAIL() << "Expected an exception"; + } catch (const VeloxException& e) { + ASSERT_TRUE(e.message().find(errorMessage) != std::string::npos) + << "Expected error message to contain '" << errorMessage << "', but received '" << e.message() << "'."; + } } std::shared_ptr veloxConvertor_ = std::make_shared(); @@ -120,7 +124,7 @@ TEST_F(VeloxSubstraitRoundTripTest, cast) { // Cast literal "abc" to int64, expecting an exception to be thrown. plan = PlanBuilder().values(vectors).project({"cast('abc' as bigint)"}).planNode(); - assertFailingPlanConversion(plan, "Failed to cast from VARCHAR to BIGINT"); + assertFailingPlanConversion(plan, "Cannot cast VARCHAR 'abc' to BIGINT."); } TEST_F(VeloxSubstraitRoundTripTest, filter) { @@ -385,7 +389,7 @@ TEST_F(VeloxSubstraitRoundTripTest, arrayLiteral) { makeConstantExpr(makeNullableArrayVector({{5.5, std::nullopt}})), makeConstantExpr(makeArrayVector({{StringView("6")}})), makeConstantExpr(makeArrayVector({{Timestamp(123'456, 123'000)}})), - makeConstantExpr(makeArrayVector({{Date(8035)}})), + makeConstantExpr(makeArrayVector({{8035}}, DATE())), makeConstantExpr(makeArrayVector({{54 * 1000}}, INTERVAL_DAY_TIME())), makeConstantExpr(makeArrayVector({{}})), // Nested array: [[1, 2, 3], [4, 5]] @@ -402,7 +406,7 @@ TEST_F(VeloxSubstraitRoundTripTest, arrayLiteral) { plan, "SELECT array[true, null], array[0, null], array[1, null], " "array[2, null], array[3, null], array[4.4, null], array[5.5, null], " - "array[6]," + "array['6']," "array['1970-01-02T10:17:36.000123000'::TIMESTAMP]," "array['1992-01-01'::DATE]," "array[INTERVAL 54 MILLISECONDS], " @@ -412,7 +416,7 @@ TEST_F(VeloxSubstraitRoundTripTest, arrayLiteral) { TEST_F(VeloxSubstraitRoundTripTest, dateType) { auto a = makeFlatVector({0, 1}); auto b = makeFlatVector({0.3, 0.4}); - auto c = makeFlatVector({Date(8036), Date(8035)}); + auto c = makeFlatVector({8036, 8035}, DATE()); auto vectors = makeRowVector({"a", "b", "c"}, {a, b, c}); createDuckDbTable({vectors}); @@ -430,15 +434,18 @@ TEST_F(VeloxSubstraitRoundTripTest, subField) { makeFlatVector({0.905, 0.968, 0.632}), }); createDuckDbTable({data}); - auto plan = - PlanBuilder() - .values({data}) - .project({"cast(row_constructor(a, b) as row(a bigint, b bigint)) as ab", "a", "b", "c"}) - .project({"cast(row_constructor(ab, c) as row(ab row(a bigint, b bigint), c bigint)) as abc", "a", "b"}) - .project({"(cast(row_constructor(a, b) as row(a bigint, b bigint))).a", "(abc).ab.a", "(abc).ab.b", "abc.c"}) - .planNode(); - - assertPlanConversion(plan, "SELECT a, a, b, c FROM tmp"); + auto plan = PlanBuilder() + .values({data}) + .project({"cast(row_constructor(a, b) as row(a bigint, b bigint)) as ab", "a", "b", "c"}) + .project({"cast(row_constructor(ab, c) as row(ab row(a bigint, b bigint), c bigint)) as abc"}) + .project({"(abc).ab.a", "(abc).ab.b", "abc.c"}) + .planNode(); + + assertPlanConversion(plan, "SELECT a, b, c FROM tmp"); + + plan = + PlanBuilder().values({data}).project({"(cast(row_constructor(a, b) as row(a bigint, b bigint))).a"}).planNode(); + assertFailingPlanConversion(plan, "Non-field expression is not supported"); } } // namespace gluten diff --git a/cpp/velox/tests/VeloxSubstraitSignatureTest.cc b/cpp/velox/tests/VeloxSubstraitSignatureTest.cc index 770d042312ed..fbfe14f7c92c 100644 --- a/cpp/velox/tests/VeloxSubstraitSignatureTest.cc +++ b/cpp/velox/tests/VeloxSubstraitSignatureTest.cc @@ -30,8 +30,12 @@ class VeloxSubstraitSignatureTest : public ::testing::Test { functions::prestosql::registerAllScalarFunctions(); } + static TypePtr fromSubstraitSignature(const std::string& signature) { + return VeloxSubstraitSignature::fromSubstraitSignature(signature); + } + static std::string toSubstraitSignature(const TypePtr& type) { - return VeloxSubstraitSignature::toSubstraitSignature(type->kind()); + return VeloxSubstraitSignature::toSubstraitSignature(type); } static std::string toSubstraitSignature(const std::string& functionName, const std::vector& arguments) { @@ -91,4 +95,50 @@ TEST_F(VeloxSubstraitSignatureTest, toSubstraitSignatureWithFunctionNameAndArgum ASSERT_ANY_THROW(toSubstraitSignature("transform_keys", std::move(types))); } +TEST_F(VeloxSubstraitSignatureTest, fromSubstraitSignature) { + ASSERT_EQ(fromSubstraitSignature("bool")->kind(), TypeKind::BOOLEAN); + ASSERT_EQ(fromSubstraitSignature("i8")->kind(), TypeKind::TINYINT); + ASSERT_EQ(fromSubstraitSignature("i16")->kind(), TypeKind::SMALLINT); + ASSERT_EQ(fromSubstraitSignature("i32")->kind(), TypeKind::INTEGER); + ASSERT_EQ(fromSubstraitSignature("i64")->kind(), TypeKind::BIGINT); + ASSERT_EQ(fromSubstraitSignature("fp32")->kind(), TypeKind::REAL); + ASSERT_EQ(fromSubstraitSignature("fp64")->kind(), TypeKind::DOUBLE); + ASSERT_EQ(fromSubstraitSignature("str")->kind(), TypeKind::VARCHAR); + ASSERT_EQ(fromSubstraitSignature("vbin")->kind(), TypeKind::VARBINARY); + ASSERT_EQ(fromSubstraitSignature("ts")->kind(), TypeKind::TIMESTAMP); + ASSERT_EQ(fromSubstraitSignature("date")->kind(), TypeKind::INTEGER); + ASSERT_EQ(fromSubstraitSignature("dec<18,2>")->kind(), TypeKind::BIGINT); + ASSERT_EQ(fromSubstraitSignature("dec<19,2>")->kind(), TypeKind::HUGEINT); + + // Struct type test. + auto type = fromSubstraitSignature("struct>"); + ASSERT_EQ(type->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->kind(), TypeKind::BOOLEAN); + ASSERT_EQ(type->childAt(1)->kind(), TypeKind::VARBINARY); + ASSERT_EQ(type->childAt(2)->kind(), TypeKind::BIGINT); + type = fromSubstraitSignature("struct>"); + ASSERT_EQ(type->childAt(1)->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(1)->childAt(0)->kind(), TypeKind::TINYINT); + ASSERT_EQ(type->childAt(1)->childAt(1)->kind(), TypeKind::REAL); + type = fromSubstraitSignature("struct,vbin,ts,dec<9,2>>"); + ASSERT_EQ(type->childAt(1)->kind(), TypeKind::ROW); + type = fromSubstraitSignature("struct,i16>"); + ASSERT_EQ(type->childAt(0)->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->childAt(0)->kind(), TypeKind::TIMESTAMP); + type = fromSubstraitSignature("struct>>"); + ASSERT_EQ(type->childAt(0)->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->childAt(0)->kind(), TypeKind::BIGINT); + type = fromSubstraitSignature("struct>>"); + ASSERT_EQ(type->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->childAt(0)->kind(), TypeKind::ROW); + ASSERT_EQ(type->childAt(0)->childAt(0)->childAt(0)->kind(), TypeKind::TINYINT); + type = fromSubstraitSignature("struct>>"); + ASSERT_EQ(type->childAt(0)->childAt(0)->childAt(0)->kind(), TypeKind::TINYINT); + ASSERT_EQ(type->childAt(0)->childAt(0)->childAt(1)->kind(), TypeKind::VARCHAR); + type = fromSubstraitSignature("struct>>>"); + ASSERT_EQ(type->childAt(0)->childAt(0)->childAt(1)->kind(), TypeKind::HUGEINT); + ASSERT_ANY_THROW(fromSubstraitSignature("other")->kind()); +} + } // namespace gluten diff --git a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc index c9b95c711332..bc9e9df05c05 100644 --- a/cpp/velox/tests/VeloxToSubstraitTypeTest.cc +++ b/cpp/velox/tests/VeloxToSubstraitTypeTest.cc @@ -32,7 +32,7 @@ class VeloxToSubstraitTypeTest : public ::testing::Test { google::protobuf::Arena arena; auto substraitType = typeConvertor_->toSubstraitType(arena, type); - auto sameType = substraitTypeToVeloxType(substraitType); + auto sameType = SubstraitParser::parseType(substraitType); ASSERT_TRUE(sameType->kindEquals(type)) << "Expected: " << type->toString() << ", but got: " << sameType->toString(); } @@ -59,6 +59,6 @@ TEST_F(VeloxToSubstraitTypeTest, basic) { testTypeConversion(ROW({"a", "b", "c"}, {BIGINT(), BOOLEAN(), VARCHAR()})); testTypeConversion(ROW({"a", "b", "c"}, {BIGINT(), ROW({"x", "y"}, {BOOLEAN(), VARCHAR()}), REAL()})); - ASSERT_ANY_THROW(testTypeConversion(ROW({}, {}))); + testTypeConversion(ROW({}, {})); } } // namespace gluten diff --git a/cpp/velox/udf/UdfLoader.cc b/cpp/velox/udf/UdfLoader.cc index fb23e8959bf8..96996ed27cb4 100644 --- a/cpp/velox/udf/UdfLoader.cc +++ b/cpp/velox/udf/UdfLoader.cc @@ -17,9 +17,10 @@ #include #include -#include #include #include +#include +#include #include #include "substrait/VeloxToSubstraitType.h" @@ -42,7 +43,7 @@ void* loadSymFromLibrary(void* handle, const std::string& libPath, const std::st } // namespace void gluten::UdfLoader::loadUdfLibraries(const std::string& libPaths) { - const auto& paths = splitPaths(libPaths); + const auto& paths = splitPaths(libPaths, /*checkExists=*/true); loadUdfLibraries0(paths); } @@ -71,7 +72,7 @@ std::unordered_map gluten::UdfLoader::getUdfMap() { auto getUdfEntries = reinterpret_cast(getUdfEntriesSym); getUdfEntries(udfEntry); - facebook::velox::dwio::type::fbhive::HiveTypeParser parser; + facebook::velox::type::fbhive::HiveTypeParser parser; google::protobuf::Arena arena; auto typeConverter = VeloxToSubstraitTypeConvertor(); for (auto i = 0; i < numUdf; ++i) { diff --git a/cpp/velox/udf/examples/TestMyUDF.cc b/cpp/velox/udf/examples/TestMyUDF.cc index 08f1610bb247..e01c2f2c9ae1 100644 --- a/cpp/velox/udf/examples/TestMyUDF.cc +++ b/cpp/velox/udf/examples/TestMyUDF.cc @@ -29,7 +29,9 @@ int main() { const std::string funcName = "myudf1"; auto f = map.withRLock([&funcName](auto& self) -> std::shared_ptr { auto iter = self.find(funcName); - return iter->second.factory(funcName, {}); + std::unordered_map values; + const facebook::velox::core::QueryConfig config(std::move(values)); + return iter->second.factory(funcName, {}, config); }); if (!f) { diff --git a/cpp/velox/utils/Common.h b/cpp/velox/utils/Common.h index 906383806124..eaa551da0566 100644 --- a/cpp/velox/utils/Common.h +++ b/cpp/velox/utils/Common.h @@ -22,6 +22,7 @@ #include #include "velox/common/base/SimdUtil.h" +#include "velox/common/time/CpuWallTimer.h" namespace gluten { @@ -34,15 +35,17 @@ static inline void fastCopy(void* dst, const void* src, size_t n) { facebook::velox::simd::memcpy(dst, src, n); } -#define START_TIMING(timing) \ - { \ - auto ptiming = &timing; \ - facebook::velox::DeltaCpuWallTimer timer{[ptiming](const CpuWallTiming& delta) { ptiming->add(delta); }}; +#define START_TIMING(timing) \ + { \ + auto ptiming = &timing; \ + facebook::velox::DeltaCpuWallTimer timer{ \ + [ptiming](const facebook::velox::CpuWallTiming& delta) { ptiming->add(delta); }}; #define END_TIMING() } -#define SCOPED_TIMER(timing) \ - auto ptiming = &timing; \ - facebook::velox::DeltaCpuWallTimer timer{[ptiming](const CpuWallTiming& delta) { ptiming->add(delta); }}; +#define SCOPED_TIMER(timing) \ + auto ptiming = &timing; \ + facebook::velox::DeltaCpuWallTimer timer{ \ + [ptiming](const facebook::velox::CpuWallTiming& delta) { ptiming->add(delta); }}; } // namespace gluten diff --git a/cpp/velox/utils/ConfigExtractor.cc b/cpp/velox/utils/ConfigExtractor.cc index 78f22a52f20d..432b9a027542 100644 --- a/cpp/velox/utils/ConfigExtractor.cc +++ b/cpp/velox/utils/ConfigExtractor.cc @@ -36,4 +36,7 @@ std::string getConfigValue( return got->second; } +bool debugModeEnabled(const std::unordered_map& confMap) { + return getConfigValue(confMap, kDebugModeEnabled, "false") == "true"; +} } // namespace gluten diff --git a/cpp/velox/utils/ConfigExtractor.h b/cpp/velox/utils/ConfigExtractor.h index 10967f946796..a755c4158aee 100644 --- a/cpp/velox/utils/ConfigExtractor.h +++ b/cpp/velox/utils/ConfigExtractor.h @@ -23,6 +23,8 @@ #include #include +#include "config/GlutenConfig.h" + namespace gluten { std::string getConfigValue( @@ -30,4 +32,6 @@ std::string getConfigValue( const std::string& key, const std::optional& fallbackValue); +bool debugModeEnabled(const std::unordered_map& confMap); + } // namespace gluten diff --git a/cpp/velox/utils/VeloxArrowUtils.cc b/cpp/velox/utils/VeloxArrowUtils.cc index 217ac06f75c3..db1458d15035 100644 --- a/cpp/velox/utils/VeloxArrowUtils.cc +++ b/cpp/velox/utils/VeloxArrowUtils.cc @@ -16,10 +16,10 @@ */ #include "utils/VeloxArrowUtils.h" - -#include -#include #include "memory/VeloxColumnarBatch.h" +#include "utils/Common.h" +#include "velox/vector/ComplexVector.h" +#include "velox/vector/arrow/Bridge.h" namespace gluten { @@ -54,48 +54,15 @@ arrow::Result> recordBatch2VeloxColumnarBatch(con return std::make_shared(std::dynamic_pointer_cast(vp)); } -arrow::Status MyMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { - if (bytes_allocated() + size > capacity_) { - return arrow::Status::OutOfMemory("malloc of size ", size, " failed"); - } - RETURN_NOT_OK(pool_->Allocate(size, alignment, out)); - stats_.UpdateAllocatedBytes(size); - return arrow::Status::OK(); -} - -arrow::Status MyMemoryPool::Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) { - if (newSize > capacity_) { - return arrow::Status::OutOfMemory("malloc of size ", newSize, " failed"); +arrow::Result> toArrowBuffer( + facebook::velox::BufferPtr buffer, + arrow::MemoryPool* pool) { + if (buffer == nullptr) { + return nullptr; } - // auto old_ptr = *ptr; - RETURN_NOT_OK(pool_->Reallocate(oldSize, newSize, alignment, ptr)); - stats_.UpdateAllocatedBytes(newSize - oldSize); - return arrow::Status::OK(); -} - -void MyMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { - pool_->Free(buffer, size, alignment); - stats_.UpdateAllocatedBytes(-size); -} - -int64_t MyMemoryPool::bytes_allocated() const { - return stats_.bytes_allocated(); -} - -int64_t MyMemoryPool::max_memory() const { - return pool_->max_memory(); -} - -int64_t MyMemoryPool::total_bytes_allocated() const { - return pool_->total_bytes_allocated(); -} - -int64_t MyMemoryPool::num_allocations() const { - throw pool_->num_allocations(); -} - -std::string MyMemoryPool::backend_name() const { - return pool_->backend_name(); + ARROW_ASSIGN_OR_RAISE(auto arrowBuffer, arrow::AllocateResizableBuffer(buffer->size(), pool)); + gluten::fastCopy(arrowBuffer->mutable_data(), buffer->asMutable(), buffer->size()); + return arrowBuffer; } } // namespace gluten diff --git a/cpp/velox/utils/VeloxArrowUtils.h b/cpp/velox/utils/VeloxArrowUtils.h index 3d1700e89ef1..763ceeb7fa6b 100644 --- a/cpp/velox/utils/VeloxArrowUtils.h +++ b/cpp/velox/utils/VeloxArrowUtils.h @@ -21,17 +21,11 @@ #include #include -#include -#include -#include -#include "memory/ArrowMemoryPool.h" #include "memory/ColumnarBatch.h" -#include "utils/macros.h" +#include "velox/buffer/Buffer.h" +#include "velox/common/memory/MemoryPool.h" #include "velox/type/Type.h" -#include -#include - namespace gluten { void toArrowSchema( @@ -45,39 +39,11 @@ std::shared_ptr toArrowSchema( facebook::velox::TypePtr fromArrowSchema(const std::shared_ptr& schema); +arrow::Result> toArrowBuffer(facebook::velox::BufferPtr buffer, arrow::MemoryPool* pool); + /** * For testing. */ arrow::Result> recordBatch2VeloxColumnarBatch(const arrow::RecordBatch& rb); -/** - * arrow::MemoryPool instance used by tests and benchmarks - */ -class MyMemoryPool final : public arrow::MemoryPool { - public: - explicit MyMemoryPool() : capacity_(std::numeric_limits::max()) {} - explicit MyMemoryPool(int64_t capacity) : capacity_(capacity) {} - - arrow::Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; - - arrow::Status Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) override; - - void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; - - int64_t bytes_allocated() const override; - - int64_t max_memory() const override; - - int64_t total_bytes_allocated() const override; - - int64_t num_allocations() const override; - - std::string backend_name() const override; - - private: - arrow::MemoryPool* pool_ = arrow::default_memory_pool(); - int64_t capacity_; - arrow::internal::MemoryPoolStats stats_; -}; - } // namespace gluten diff --git a/cpp/velox/utils/tests/LocalRssClient.h b/cpp/velox/utils/tests/LocalRssClient.h new file mode 100644 index 000000000000..2c43c5947d8e --- /dev/null +++ b/cpp/velox/utils/tests/LocalRssClient.h @@ -0,0 +1,80 @@ +/* + * 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 "shuffle/rss/RssClient.h" +#include "utils/Common.h" +#include "utils/macros.h" + +namespace gluten { + +class LocalRssClient : public RssClient { + public: + LocalRssClient(std::string dataFile) : dataFile_(dataFile) {} + + int32_t pushPartitionData(int32_t partitionId, char* bytes, int64_t size) { + auto idx = -1; + auto maybeIdx = partitionIdx_.find(partitionId); + auto returnSize = size; + if (maybeIdx == partitionIdx_.end()) { + idx = partitionIdx_.size(); + partitionIdx_[partitionId] = idx; + auto buffer = arrow::AllocateResizableBuffer(0).ValueOrDie(); + partitionBuffers_.push_back(std::move(buffer)); + // Add EOS length. + returnSize += sizeof(int32_t) * 2; + } else { + idx = maybeIdx->second; + } + + auto& buffer = partitionBuffers_[idx]; + auto newSize = buffer->size() + size; + if (buffer->capacity() < newSize) { + GLUTEN_THROW_NOT_OK(buffer->Reserve(newSize)); + } + memcpy(buffer->mutable_data() + buffer->size(), bytes, size); + GLUTEN_THROW_NOT_OK(buffer->Resize(newSize)); + return returnSize; + } + + void stop() { + std::shared_ptr fout; + GLUTEN_ASSIGN_OR_THROW(fout, arrow::io::FileOutputStream::Open(dataFile_)); + + int64_t bytes; // unused + for (auto item : partitionIdx_) { + auto idx = item.second; + GLUTEN_THROW_NOT_OK(fout->Write(partitionBuffers_[idx]->data(), partitionBuffers_[idx]->size())); + GLUTEN_THROW_NOT_OK(writeEos(fout.get(), &bytes)); + GLUTEN_THROW_NOT_OK(fout->Flush()); + } + GLUTEN_THROW_NOT_OK(fout->Close()); + partitionBuffers_.clear(); + partitionIdx_.clear(); + } + + private: + std::string dataFile_; + std::vector> partitionBuffers_; + std::map partitionIdx_; +}; + +} // namespace gluten diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.cc b/cpp/velox/utils/tests/MemoryPoolUtils.cc new file mode 100644 index 000000000000..21ca464b47fd --- /dev/null +++ b/cpp/velox/utils/tests/MemoryPoolUtils.cc @@ -0,0 +1,150 @@ +/* + * 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 "utils/tests/MemoryPoolUtils.h" + +namespace gluten { + +arrow::Status LimitedMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { + if (bytes_allocated() + size > capacity_) { + return arrow::Status::OutOfMemory("malloc of size ", size, " failed"); + } + RETURN_NOT_OK(pool_->Allocate(size, alignment, out)); + stats_.UpdateAllocatedBytes(size); + return arrow::Status::OK(); +} + +arrow::Status LimitedMemoryPool::Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) { + if (newSize > capacity_) { + return arrow::Status::OutOfMemory("malloc of size ", newSize, " failed"); + } + RETURN_NOT_OK(pool_->Reallocate(oldSize, newSize, alignment, ptr)); + stats_.UpdateAllocatedBytes(newSize - oldSize); + return arrow::Status::OK(); +} + +void LimitedMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { + pool_->Free(buffer, size, alignment); + stats_.UpdateAllocatedBytes(-size); +} + +int64_t LimitedMemoryPool::bytes_allocated() const { + return stats_.bytes_allocated(); +} + +int64_t LimitedMemoryPool::max_memory() const { + return pool_->max_memory(); +} + +int64_t LimitedMemoryPool::total_bytes_allocated() const { + return pool_->total_bytes_allocated(); +} + +int64_t LimitedMemoryPool::num_allocations() const { + throw pool_->num_allocations(); +} + +std::string LimitedMemoryPool::backend_name() const { + return pool_->backend_name(); +} + +bool SelfEvictedMemoryPool::checkEvict(int64_t newCapacity, std::function block) { + bytesEvicted_ = 0; + auto capacity = capacity_; + // Limit the capacity to trigger evict. + setCapacity(newCapacity); + + block(); + + capacity_ = capacity; + return bytesEvicted_ > 0; +} + +void SelfEvictedMemoryPool::setCapacity(int64_t capacity) { + if (capacity < bytes_allocated()) { + capacity_ = bytes_allocated(); + } else { + capacity_ = capacity; + } +} + +int64_t SelfEvictedMemoryPool::capacity() const { + return capacity_; +} + +void SelfEvictedMemoryPool::setEvictable(Evictable* evictable) { + evictable_ = evictable; +} + +arrow::Status SelfEvictedMemoryPool::Allocate(int64_t size, int64_t alignment, uint8_t** out) { + RETURN_NOT_OK(evict(size)); + return pool_->Allocate(size, alignment, out); +} + +arrow::Status SelfEvictedMemoryPool::Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) { + if (newSize > oldSize) { + RETURN_NOT_OK(evict(newSize - oldSize)); + } + return pool_->Reallocate(oldSize, newSize, alignment, ptr); +} + +void SelfEvictedMemoryPool::Free(uint8_t* buffer, int64_t size, int64_t alignment) { + return pool_->Free(buffer, size, alignment); +} + +int64_t SelfEvictedMemoryPool::bytes_allocated() const { + return pool_->bytes_allocated(); +} + +int64_t SelfEvictedMemoryPool::max_memory() const { + return pool_->max_memory(); +} + +std::string SelfEvictedMemoryPool::backend_name() const { + return pool_->backend_name(); +} + +int64_t SelfEvictedMemoryPool::total_bytes_allocated() const { + return pool_->total_bytes_allocated(); +} + +int64_t SelfEvictedMemoryPool::num_allocations() const { + throw pool_->num_allocations(); +} + +arrow::Status SelfEvictedMemoryPool::evict(int64_t size) { + VELOX_CHECK_NOT_NULL(evictable_); + if (size > capacity_ - pool_->bytes_allocated()) { + // Self evict. + int64_t actual; + RETURN_NOT_OK(evictable_->evictFixedSize(size, &actual)); + if (size > capacity_ - pool_->bytes_allocated()) { + return arrow::Status::OutOfMemory( + "Failed to allocate after evict. Capacity: ", + capacity_, + ", Requested: ", + size, + ", Evicted: ", + actual, + ", Allocated: ", + pool_->bytes_allocated()); + } + bytesEvicted_ += actual; + } + return arrow::Status::OK(); +} +} // namespace gluten \ No newline at end of file diff --git a/cpp/velox/utils/tests/MemoryPoolUtils.h b/cpp/velox/utils/tests/MemoryPoolUtils.h new file mode 100644 index 000000000000..a49fc02d7981 --- /dev/null +++ b/cpp/velox/utils/tests/MemoryPoolUtils.h @@ -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. + */ + +#pragma once + +#include +#include "memory/Evictable.h" +#include "utils/exception.h" +#include "velox/common/base/Exceptions.h" + +namespace gluten { + +/** + * 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()) {} + explicit LimitedMemoryPool(int64_t capacity) : capacity_(capacity) {} + + arrow::Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; + + arrow::Status Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) override; + + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; + + int64_t bytes_allocated() const override; + + int64_t max_memory() const override; + + int64_t total_bytes_allocated() const override; + + int64_t num_allocations() const override; + + std::string backend_name() const override; + + private: + arrow::MemoryPool* pool_ = arrow::default_memory_pool(); + int64_t capacity_; + arrow::internal::MemoryPoolStats stats_; +}; + +/** + * 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) : pool_(pool) {} + + bool checkEvict(int64_t newCapacity, std::function block); + + void setCapacity(int64_t capacity); + + int64_t capacity() const; + + void setEvictable(Evictable* evictable); + + arrow::Status Allocate(int64_t size, int64_t alignment, uint8_t** out) override; + + arrow::Status Reallocate(int64_t oldSize, int64_t newSize, int64_t alignment, uint8_t** ptr) override; + + void Free(uint8_t* buffer, int64_t size, int64_t alignment) override; + + int64_t bytes_allocated() const override; + + int64_t max_memory() const override; + + std::string backend_name() const override; + + int64_t total_bytes_allocated() const override; + + int64_t num_allocations() const override; + + private: + arrow::Status evict(int64_t size); + + arrow::MemoryPool* pool_; + Evictable* evictable_; + int64_t capacity_{std::numeric_limits::max()}; + + int64_t bytesEvicted_{0}; +}; + +} // namespace gluten diff --git a/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h new file mode 100644 index 000000000000..0fb10faac655 --- /dev/null +++ b/cpp/velox/utils/tests/VeloxShuffleWriterTestBase.h @@ -0,0 +1,409 @@ +/* + * 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 "LocalRssClient.h" +#include "memory/VeloxColumnarBatch.h" +#include "shuffle/PartitionWriter.h" +#include "shuffle/VeloxShuffleReader.h" +#include "utils/Compression.h" +#include "velox/vector/tests/VectorTestUtils.h" + +namespace gluten { + +struct ShuffleTestParams { + PartitionWriterType partition_writer_type; + arrow::Compression::type compression_type; + CompressionMode compression_mode; + + std::string toString() const { + std::ostringstream out; + out << "partition_writer_type = " << partition_writer_type << "compression_type = " << compression_type + << ", compression_mode = " << compression_mode; + return out.str(); + } +}; + +class VeloxShuffleWriterTestBase : public facebook::velox::test::VectorTestBase { + protected: + void setUp() { + shuffleWriterOptions_ = ShuffleWriterOptions::defaults(); + shuffleWriterOptions_.compression_threshold = 0; + shuffleWriterOptions_.memory_pool = defaultArrowMemoryPool().get(); + GLUTEN_THROW_NOT_OK(setLocalDirsAndDataFile()); + + // Set up test data. + children1_ = { + makeNullableFlatVector({1, 2, 3, std::nullopt, 4, std::nullopt, 5, 6, std::nullopt, 7}), + makeNullableFlatVector({1, -1, std::nullopt, std::nullopt, -2, 2, std::nullopt, std::nullopt, 3, -3}), + makeNullableFlatVector({1, 2, 3, 4, std::nullopt, 5, 6, 7, 8, std::nullopt}), + makeNullableFlatVector( + {std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt, + std::nullopt}), + makeNullableFlatVector( + {-0.1234567, + std::nullopt, + 0.1234567, + std::nullopt, + -0.142857, + std::nullopt, + 0.142857, + 0.285714, + 0.428617, + std::nullopt}), + makeNullableFlatVector( + {std::nullopt, true, false, std::nullopt, true, true, false, true, std::nullopt, std::nullopt}), + makeFlatVector( + {"alice0", "bob1", "alice2", "bob3", "Alice4", "Bob5", "AlicE6", "boB7", "ALICE8", "BOB9"}), + makeNullableFlatVector( + {"alice", "bob", std::nullopt, std::nullopt, "Alice", "Bob", std::nullopt, "alicE", std::nullopt, "boB"}), + }; + + children2_ = { + makeNullableFlatVector({std::nullopt, std::nullopt}), + makeFlatVector({1, -1}), + makeNullableFlatVector({100, std::nullopt}), + makeFlatVector({1, 1}), + makeFlatVector({0.142857, -0.142857}), + makeFlatVector({true, false}), + makeFlatVector( + {"bob", + "alicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealicealice"}), + makeNullableFlatVector({std::nullopt, std::nullopt}), + }; + + childrenNoNull_ = { + makeFlatVector({0, 1}), + makeFlatVector({0, -1}), + makeFlatVector({0, 100}), + makeFlatVector({0, 1}), + makeFlatVector({0, 0.142857}), + makeFlatVector({false, true}), + makeFlatVector({"", "alice"}), + makeFlatVector({"alice", ""}), + }; + + inputVector1_ = makeRowVector(children1_); + inputVector2_ = makeRowVector(children2_); + inputVectorNoNull_ = makeRowVector(childrenNoNull_); + } + + arrow::Status splitRowVector(VeloxShuffleWriter& shuffleWriter, facebook::velox::RowVectorPtr vector) { + std::shared_ptr cb = std::make_shared(vector); + return shuffleWriter.split(cb, ShuffleWriter::kMinMemLimit); + } + + // Create multiple local dirs and join with comma. + arrow::Status setLocalDirsAndDataFile() { + auto& localDirs = shuffleWriterOptions_.local_dirs; + static const std::string kTestLocalDirsPrefix = "columnar-shuffle-test-"; + + // Create first tmp dir and create data file. + // To prevent tmpDirs from being deleted in the dtor, we need to store them. + tmpDirs_.emplace_back(); + ARROW_ASSIGN_OR_RAISE(tmpDirs_.back(), arrow::internal::TemporaryDir::Make(kTestLocalDirsPrefix)) + ARROW_ASSIGN_OR_RAISE(shuffleWriterOptions_.data_file, createTempShuffleFile(tmpDirs_.back()->path().ToString())); + localDirs += tmpDirs_.back()->path().ToString(); + localDirs.push_back(','); + + // Create second tmp dir. + tmpDirs_.emplace_back(); + ARROW_ASSIGN_OR_RAISE(tmpDirs_.back(), arrow::internal::TemporaryDir::Make(kTestLocalDirsPrefix)) + localDirs += tmpDirs_.back()->path().ToString(); + return arrow::Status::OK(); + } + + virtual std::shared_ptr createShuffleWriter() = 0; + + ShuffleWriterOptions shuffleWriterOptions_; + + std::shared_ptr partitionWriterCreator_; + + std::vector> tmpDirs_; + + std::vector children1_; + std::vector children2_; + std::vector childrenNoNull_; + + facebook::velox::RowVectorPtr inputVector1_; + facebook::velox::RowVectorPtr inputVector2_; + facebook::velox::RowVectorPtr inputVectorNoNull_; +}; + +class VeloxShuffleWriterTest : public ::testing::TestWithParam, public VeloxShuffleWriterTestBase { + protected: + virtual void SetUp() override { + VeloxShuffleWriterTestBase::setUp(); + + ShuffleTestParams params = GetParam(); + if (params.partition_writer_type == PartitionWriterType::kCeleborn) { + partitionWriterCreator_ = std::make_shared( + std::make_shared(shuffleWriterOptions_.data_file)); + shuffleWriterOptions_.partition_writer_type = kCeleborn; + } else { + partitionWriterCreator_ = std::make_shared(); + } + shuffleWriterOptions_.compression_type = params.compression_type; + shuffleWriterOptions_.compression_mode = params.compression_mode; + } + + void TearDown() override { + if (file_ != nullptr && !file_->closed()) { + GLUTEN_THROW_NOT_OK(file_->Close()); + } + } + + static void checkFileExists(const std::string& fileName) { + ASSERT_EQ(*arrow::internal::FileExists(*arrow::internal::PlatformFilename::FromString(fileName)), true); + } + + std::shared_ptr getArrowSchema(facebook::velox::RowVectorPtr& rowVector) { + return toArrowSchema(rowVector->type(), pool()); + } + + void setReadableFile(const std::string& fileName) { + if (file_ != nullptr && !file_->closed()) { + GLUTEN_THROW_NOT_OK(file_->Close()); + } + GLUTEN_ASSIGN_OR_THROW(file_, arrow::io::ReadableFile::Open(fileName)) + } + + void getRowVectors(std::shared_ptr schema, std::vector& vectors) { + ShuffleReaderOptions options; + options.compression_type = shuffleWriterOptions_.compression_type; + auto reader = std::make_shared(schema, options, defaultArrowMemoryPool().get(), pool_); + auto iter = reader->readStream(file_); + while (iter->hasNext()) { + auto vector = std::dynamic_pointer_cast(iter->next())->getRowVector(); + vectors.emplace_back(vector); + } + } + + std::shared_ptr file_; +}; + +class SinglePartitioningShuffleWriter : public VeloxShuffleWriterTest { + protected: + void testShuffleWrite(VeloxShuffleWriter& shuffleWriter, std::vector vectors) { + for (auto& vector : vectors) { + ASSERT_NOT_OK(splitRowVector(shuffleWriter, vector)); + // No partition buffers for single partitioner. + ASSERT_EQ(shuffleWriter.partitionBufferSize(), 0); + } + ASSERT_NOT_OK(shuffleWriter.stop()); + // verify data file + checkFileExists(shuffleWriter.dataFile()); + // verify output temporary files + const auto& lengths = shuffleWriter.partitionLengths(); + ASSERT_EQ(lengths.size(), 1); + + auto schema = getArrowSchema(vectors[0]); + std::vector deserializedVectors; + setReadableFile(shuffleWriter.dataFile()); + getRowVectors(schema, deserializedVectors); + + ASSERT_EQ(deserializedVectors.size(), vectors.size()); + for (int32_t i = 0; i < deserializedVectors.size(); i++) { + facebook::velox::test::assertEqualVectors(vectors[i], deserializedVectors[i]); + } + } + + std::shared_ptr createShuffleWriter() override { + shuffleWriterOptions_.buffer_size = 10; + shuffleWriterOptions_.partitioning = Partitioning::kSingle; + GLUTEN_ASSIGN_OR_THROW( + auto shuffleWriter, VeloxShuffleWriter::create(1, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + return shuffleWriter; + } +}; + +class MultiplePartitioningShuffleWriter : public VeloxShuffleWriterTest { + protected: + void shuffleWriteReadMultiBlocks( + VeloxShuffleWriter& shuffleWriter, + int32_t expectPartitionLength, + facebook::velox::TypePtr dataType, + std::vector> expectedVectors) { /* blockId = pid, rowVector in block */ + ASSERT_NOT_OK(shuffleWriter.stop()); + // verify data file + checkFileExists(shuffleWriter.dataFile()); + // verify output temporary files + const auto& lengths = shuffleWriter.partitionLengths(); + ASSERT_EQ(lengths.size(), expectPartitionLength); + int64_t lengthSum = std::accumulate(lengths.begin(), lengths.end(), 0); + auto schema = toArrowSchema(dataType, pool()); + setReadableFile(shuffleWriter.dataFile()); + ASSERT_EQ(*file_->GetSize(), lengthSum); + for (int32_t i = 0; i < expectPartitionLength; i++) { + if (expectedVectors[i].size() == 0) { + ASSERT_EQ(lengths[i], 0); + } else { + std::vector deserializedVectors; + getRowVectors(schema, deserializedVectors); + if (i != 0) { + ASSERT_NOT_OK(file_->Advance(lengths[i - 1])); + } + ASSERT_EQ(expectedVectors[i].size(), deserializedVectors.size()); + for (int32_t j = 0; j < expectedVectors[i].size(); j++) { + facebook::velox::test::assertEqualVectors(expectedVectors[i][j], deserializedVectors[j]); + } + } + } + } + + void testShuffleWriteMultiBlocks( + VeloxShuffleWriter& shuffleWriter, + std::vector vectors, + int32_t expectPartitionLength, + facebook::velox::TypePtr dataType, + std::vector> expectedVectors) { + for (auto& vector : vectors) { + ASSERT_NOT_OK(splitRowVector(shuffleWriter, vector)); + } + shuffleWriteReadMultiBlocks(shuffleWriter, expectPartitionLength, dataType, expectedVectors); + } +}; + +class HashPartitioningShuffleWriter : public MultiplePartitioningShuffleWriter { + protected: + void SetUp() override { + MultiplePartitioningShuffleWriter::SetUp(); + + children1_.insert((children1_.begin()), makeFlatVector({1, 2, 2, 2, 2, 1, 1, 1, 2, 1})); + hashInputVector1_ = makeRowVector(children1_); + children2_.insert((children2_.begin()), makeFlatVector({2, 2})); + hashInputVector2_ = makeRowVector(children2_); + } + + std::shared_ptr createShuffleWriter() override { + shuffleWriterOptions_.buffer_size = 4; + shuffleWriterOptions_.partitioning = Partitioning::kHash; + GLUTEN_ASSIGN_OR_THROW( + auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + return shuffleWriter; + } + + std::vector hashPartitionIds_{1, 2}; + + facebook::velox::RowVectorPtr hashInputVector1_; + facebook::velox::RowVectorPtr hashInputVector2_; +}; + +class RangePartitioningShuffleWriter : public MultiplePartitioningShuffleWriter { + protected: + void SetUp() override { + MultiplePartitioningShuffleWriter::SetUp(); + + auto pid1 = makeRowVector({makeFlatVector({0, 1, 0, 1, 0, 1, 0, 1, 0, 1})}); + auto rangeVector1 = makeRowVector(inputVector1_->children()); + compositeBatch1_ = CompositeColumnarBatch::create( + {std::make_shared(pid1), std::make_shared(rangeVector1)}); + + auto pid2 = makeRowVector({makeFlatVector({0, 1})}); + auto rangeVector2 = makeRowVector(inputVector2_->children()); + compositeBatch2_ = CompositeColumnarBatch::create( + {std::make_shared(pid2), std::make_shared(rangeVector2)}); + } + + std::shared_ptr createShuffleWriter() override { + shuffleWriterOptions_.buffer_size = 4; + shuffleWriterOptions_.partitioning = Partitioning::kRange; + GLUTEN_ASSIGN_OR_THROW( + auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + return shuffleWriter; + } + + void testShuffleWriteMultiBlocks( + VeloxShuffleWriter& shuffleWriter, + std::vector> batches, + int32_t expectPartitionLength, + facebook::velox::TypePtr dataType, + std::vector> expectedVectors) { /* blockId = pid, rowVector in block */ + for (auto& batch : batches) { + ASSERT_NOT_OK(shuffleWriter.split(batch, ShuffleWriter::kMinMemLimit)); + } + shuffleWriteReadMultiBlocks(shuffleWriter, expectPartitionLength, dataType, expectedVectors); + } + + std::shared_ptr compositeBatch1_; + std::shared_ptr compositeBatch2_; +}; + +class RoundRobinPartitioningShuffleWriter : public MultiplePartitioningShuffleWriter { + protected: + std::shared_ptr createShuffleWriter() override { + shuffleWriterOptions_.buffer_size = 4; + GLUTEN_ASSIGN_OR_THROW( + auto shuffleWriter, VeloxShuffleWriter::create(2, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + return shuffleWriter; + } +}; + +class VeloxShuffleWriterMemoryTest : public VeloxShuffleWriterTestBase, public testing::Test { + protected: + void SetUp() override { + VeloxShuffleWriterTestBase::setUp(); + // Use LocalPartitionWriter to test OOM and spill. + partitionWriterCreator_ = std::make_shared(); + } + + std::shared_ptr createShuffleWriter(uint32_t numPartitions) { + GLUTEN_ASSIGN_OR_THROW( + auto shuffleWriter, + VeloxShuffleWriter::create(numPartitions, partitionWriterCreator_, shuffleWriterOptions_, pool_)) + return shuffleWriter; + } + + std::shared_ptr createShuffleWriter() override { + return createShuffleWriter(kDefaultShufflePartitions); + } + + int64_t splitRowVectorAndSpill( + VeloxShuffleWriter& shuffleWriter, + std::vector vectors, + bool shrink) { + for (auto vector : vectors) { + ASSERT_NOT_OK(splitRowVector(shuffleWriter, vector)); + } + + auto targetEvicted = shuffleWriter.cachedPayloadSize(); + if (shrink) { + targetEvicted += shuffleWriter.partitionBufferSize(); + } + int64_t evicted; + ASSERT_NOT_OK(shuffleWriter.evictFixedSize(targetEvicted, &evicted)); + + return evicted; + }; + + static constexpr uint32_t kDefaultShufflePartitions = 2; +}; + +} // namespace gluten diff --git a/dev/buildbundle-veloxbe.sh b/dev/buildbundle-veloxbe.sh index ca78ddaaee13..3bfd6994a556 100755 --- a/dev/buildbundle-veloxbe.sh +++ b/dev/buildbundle-veloxbe.sh @@ -6,3 +6,4 @@ source "$BASEDIR/builddeps-veloxbe.sh" cd $GLUTEN_DIR mvn clean package -Pbackends-velox -Prss -Pspark-3.2 -DskipTests mvn clean package -Pbackends-velox -Prss -Pspark-3.3 -DskipTests +mvn clean package -Pbackends-velox -Prss -Pspark-3.4 -DskipTests diff --git a/dev/builddeps-veloxbe.sh b/dev/builddeps-veloxbe.sh index 0493ec4b43fd..588221d28aef 100755 --- a/dev/builddeps-veloxbe.sh +++ b/dev/builddeps-veloxbe.sh @@ -17,6 +17,7 @@ BUILD_PROTOBUF=ON ENABLE_QAT=OFF ENABLE_IAA=OFF ENABLE_HBM=OFF +ENABLE_GCS=OFF ENABLE_S3=OFF ENABLE_HDFS=OFF ENABLE_EP_CACHE=OFF @@ -65,6 +66,10 @@ do BUILD_PROTOBUF=("${arg#*=}") shift # Remove argument name from processing ;; + --enable_gcs=*) + ENABLE_GCS=("${arg#*=}") + shift # Remove argument name from processing + ;; --enable_s3=*) ENABLE_S3=("${arg#*=}") shift # Remove argument name from processing @@ -98,19 +103,12 @@ if [ "$ENABLE_VCPKG" = "ON" ]; then eval "$envs" fi -##install arrow -if [ "$SKIP_BUILD_EP" != "ON" ]; then - cd $GLUTEN_DIR/ep/build-arrow/src - ./get_arrow.sh --enable_custom_codec=$ARROW_ENABLE_CUSTOM_CODEC - ./build_arrow.sh --build_type=$BUILD_TYPE --enable_ep_cache=$ENABLE_EP_CACHE -fi - ##install velox if [ "$SKIP_BUILD_EP" != "ON" ]; then cd $GLUTEN_DIR/ep/build-velox/src - ./get_velox.sh --enable_hdfs=$ENABLE_HDFS --build_protobuf=$BUILD_PROTOBUF --enable_s3=$ENABLE_S3 - ./build_velox.sh --enable_s3=$ENABLE_S3 --build_type=$BUILD_TYPE --enable_hdfs=$ENABLE_HDFS \ - --enable_ep_cache=$ENABLE_EP_CACHE --build_benchmarks=$BUILD_BENCHMARKS + ./get_velox.sh --enable_hdfs=$ENABLE_HDFS --build_protobuf=$BUILD_PROTOBUF --enable_s3=$ENABLE_S3 --enable_gcs=$ENABLE_GCS + ./build_velox.sh --enable_s3=$ENABLE_S3 --enable_gcs=$ENABLE_GCS --build_type=$BUILD_TYPE --enable_hdfs=$ENABLE_HDFS \ + --enable_ep_cache=$ENABLE_EP_CACHE --build_tests=$BUILD_TESTS --build_benchmarks=$BUILD_BENCHMARKS fi ## compile gluten cpp @@ -120,5 +118,5 @@ mkdir build cd build cmake -DBUILD_VELOX_BACKEND=ON -DCMAKE_BUILD_TYPE=$BUILD_TYPE \ -DBUILD_TESTS=$BUILD_TESTS -DBUILD_EXAMPLES=$BUILD_EXAMPLES -DBUILD_BENCHMARKS=$BUILD_BENCHMARKS -DBUILD_JEMALLOC=$BUILD_JEMALLOC \ - -DENABLE_HBM=$ENABLE_HBM -DENABLE_QAT=$ENABLE_QAT -DENABLE_IAA=$ENABLE_IAA -DENABLE_S3=$ENABLE_S3 -DENABLE_HDFS=$ENABLE_HDFS .. + -DENABLE_HBM=$ENABLE_HBM -DENABLE_QAT=$ENABLE_QAT -DENABLE_IAA=$ENABLE_IAA -DENABLE_GCS=$ENABLE_GCS -DENABLE_S3=$ENABLE_S3 -DENABLE_HDFS=$ENABLE_HDFS .. make -j diff --git a/dev/package.sh b/dev/package.sh index b5dc02692e17..c06a3cdf04b7 100755 --- a/dev/package.sh +++ b/dev/package.sh @@ -12,6 +12,7 @@ VERSION=$(. /etc/os-release && echo ${VERSION_ID}) $GLUTEN_DIR/dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --enable_s3=ON --enable_hdfs=ON mvn clean package -Pbackends-velox -Prss -Pspark-3.2 -DskipTests mvn clean package -Pbackends-velox -Prss -Pspark-3.3 -DskipTests +mvn clean package -Pbackends-velox -Prss -Pspark-3.4 -DskipTests mkdir -p $THIRDPARTY_LIB function process_setup_ubuntu_2004 { diff --git a/dev/vcpkg/init.sh b/dev/vcpkg/init.sh index 7316db327b6f..1b481ccb0883 100755 --- a/dev/vcpkg/init.sh +++ b/dev/vcpkg/init.sh @@ -12,7 +12,7 @@ VCPKG_TRIPLET=x64-linux-avx cd "$SCRIPT_ROOT" if [ ! -d "$VCPKG_ROOT" ] || [ -z "$(ls "$VCPKG_ROOT")" ]; then - git clone https://github.com/microsoft/vcpkg.git --branch 2023.04.15 "$VCPKG_ROOT" + git clone https://github.com/microsoft/vcpkg.git --branch master "$VCPKG_ROOT" fi [ -f "$VCPKG" ] || "$VCPKG_ROOT/bootstrap-vcpkg.sh" -disableMetrics @@ -47,4 +47,4 @@ if [ "\${GLUTEN_VCPKG_ENABLED:-}" != "${VCPKG_ROOT}" ]; then else echo "Gluten's vcpkg environment is enabled" >&2 fi -EOF \ No newline at end of file +EOF diff --git a/dev/vcpkg/ports/grpc/00001-fix-uwp.patch b/dev/vcpkg/ports/grpc/00001-fix-uwp.patch new file mode 100644 index 000000000000..2aea0afd765e --- /dev/null +++ b/dev/vcpkg/ports/grpc/00001-fix-uwp.patch @@ -0,0 +1,73 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 25990a5d8a..4bec4e1e72 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -219,6 +219,9 @@ if(UNIX) + endif() + if(WIN32) + set(_gRPC_PLATFORM_WINDOWS ON) ++ if(CMAKE_SYSTEM_NAME STREQUAL "WindowsStore") ++ set(_gRPC_PLATFORM_UWP ON) ++ endif() + endif() + + # Use C11 standard +@@ -263,6 +266,9 @@ if(MSVC) + set(_gRPC_C_CXX_FLAGS "${_gRPC_C_CXX_FLAGS} /wd4267") + # TODO(jtattermusch): needed to build boringssl with VS2017, revisit later + set(_gRPC_C_CXX_FLAGS "${_gRPC_C_CXX_FLAGS} /wd4987 /wd4774 /wd4819 /wd4996 /wd4619") ++ if(_gRPC_PLATFORM_UWP) ++ add_definitions(-DGRPC_ARES=0) ++ endif() + # Silences thousands of trucation warnings + set(_gRPC_C_CXX_FLAGS "${_gRPC_C_CXX_FLAGS} /wd4503") + # Tell MSVC to build grpc using utf-8 +@@ -430,6 +436,10 @@ file(MAKE_DIRECTORY ${_gRPC_PROTO_GENS_DIR}) + # ``.proto`` files + # + function(protobuf_generate_grpc_cpp) ++ if(_gRPC_PLATFORM_UWP) ++ return() ++ endif() ++ + if(NOT ARGN) + message(SEND_ERROR "Error: PROTOBUF_GENERATE_GRPC_CPP() called without any proto files") + return() +@@ -552,6 +562,7 @@ if (gRPC_BUILD_GRPC_RUBY_PLUGIN) + list(APPEND _gRPC_PLUGIN_LIST grpc_ruby_plugin) + endif () + ++if(NOT _gRPC_PLATFORM_UWP) + add_custom_target(plugins + DEPENDS ${_gRPC_PLUGIN_LIST} + ) +@@ -567,6 +578,7 @@ add_custom_target(tools_cxx + + add_custom_target(tools + DEPENDS tools_c tools_cxx) ++endif() + + protobuf_generate_grpc_cpp_with_import_path_correction( + src/proto/grpc/channelz/channelz.proto src/proto/grpc/channelz/channelz.proto +diff --git a/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc b/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc +index 5d2bdc14de..e9870c2656 100644 +--- a/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc ++++ b/src/core/lib/security/credentials/alts/check_gcp_environment_windows.cc +@@ -39,6 +39,7 @@ bool check_bios_data(const char*) { return false; } + bool check_windows_registry_product_name(HKEY root_key, + const char* reg_key_path, + const char* reg_key_name) { ++#if !defined(WINAPI_FAMILY) || WINAPI_FAMILY == WINAPI_FAMILY_DESKTOP_APP + const size_t kProductNameBufferSize = 256; + char const expected_substr[] = "Google"; + +@@ -71,6 +72,9 @@ bool check_windows_registry_product_name(HKEY root_key, + } + + return strstr(buffer, expected_substr) != nullptr; ++#else ++ return false; ++#endif + } + + } // namespace internal diff --git a/dev/vcpkg/ports/grpc/00002-static-linking-in-linux.patch b/dev/vcpkg/ports/grpc/00002-static-linking-in-linux.patch new file mode 100644 index 000000000000..8999a24aea09 --- /dev/null +++ b/dev/vcpkg/ports/grpc/00002-static-linking-in-linux.patch @@ -0,0 +1,16 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 8f78306f77..e09f8fcc1e 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -211,6 +211,11 @@ if (NOT DEFINED CMAKE_POSITION_INDEPENDENT_CODE) + endif() + list(APPEND CMAKE_MODULE_PATH "${CMAKE_CURRENT_SOURCE_DIR}/cmake/modules") + ++if (gRPC_STATIC_LINKING AND NOT _gRPC_PLATFORM_WINDOWS) ++ # Force to static link ++ set(CMAKE_EXE_LINKER_FLAGS "-Bstatic") ++endif() ++ + if(MSVC) + include(cmake/msvc_static_runtime.cmake) + add_definitions(-D_WIN32_WINNT=0x600 -D_SCL_SECURE_NO_WARNINGS -D_CRT_SECURE_NO_WARNINGS -D_WINSOCK_DEPRECATED_NO_WARNINGS) diff --git a/dev/vcpkg/ports/grpc/00003-undef-base64-macro.patch b/dev/vcpkg/ports/grpc/00003-undef-base64-macro.patch new file mode 100644 index 000000000000..be182659d410 --- /dev/null +++ b/dev/vcpkg/ports/grpc/00003-undef-base64-macro.patch @@ -0,0 +1,13 @@ +diff --git a/src/core/lib/transport/transport.cc b/src/core/lib/transport/transport.cc +index 5d74ec2..89494b5 100644 +--- a/src/core/lib/transport/transport.cc ++++ b/src/core/lib/transport/transport.cc +@@ -77,6 +77,8 @@ void grpc_stream_ref_init(grpc_stream_refcount* refcount, int /*initial_refs*/, + : nullptr); + } + ++#undef move64 ++ + static void move64bits(uint64_t* from, uint64_t* to) { + *to += *from; + *from = 0; diff --git a/dev/vcpkg/ports/grpc/00004-link-gdi32-on-windows.patch b/dev/vcpkg/ports/grpc/00004-link-gdi32-on-windows.patch new file mode 100644 index 000000000000..54b55c60b89d --- /dev/null +++ b/dev/vcpkg/ports/grpc/00004-link-gdi32-on-windows.patch @@ -0,0 +1,13 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 25990a5d8a..8a632d2289 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -404,7 +404,7 @@ if (NOT EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/third_party/xds) + endif() + + if(WIN32) +- set(_gRPC_BASELIB_LIBRARIES ws2_32 crypt32) ++ set(_gRPC_BASELIB_LIBRARIES wsock32 ws2_32 crypt32 gdi32) + endif() + + # Create directory for proto source files diff --git a/dev/vcpkg/ports/grpc/00005-fix-uwp-error.patch b/dev/vcpkg/ports/grpc/00005-fix-uwp-error.patch new file mode 100644 index 000000000000..23f885d48e48 --- /dev/null +++ b/dev/vcpkg/ports/grpc/00005-fix-uwp-error.patch @@ -0,0 +1,47 @@ +diff --git a/src/core/ext/transport/chttp2/transport/hpack_parser.cc b/src/core/ext/transport/chttp2/transport/hpack_parser.cc +index b0d3a6465b..5c48f1aa30 100644 +--- a/src/core/ext/transport/chttp2/transport/hpack_parser.cc ++++ b/src/core/ext/transport/chttp2/transport/hpack_parser.cc +@@ -1037,7 +1037,7 @@ class HPackParser::Parser { + + private: + void GPR_ATTRIBUTE_NOINLINE LogHeader(const HPackTable::Memento& memento) { +- const char* type; ++ const char* type = nullptr; + switch (log_info_.type) { + case LogInfo::kHeaders: + type = "HDR"; +diff --git a/src/core/lib/slice/slice.cc b/src/core/lib/slice/slice.cc +index 898a62823c..6b31cdc082 100644 +--- a/src/core/lib/slice/slice.cc ++++ b/src/core/lib/slice/slice.cc +@@ -188,6 +188,7 @@ grpc_slice grpc_slice_from_moved_buffer(grpc_core::UniquePtr p, + size_t len) { + uint8_t* ptr = reinterpret_cast(p.get()); + grpc_slice slice; ++ memset(&slice, 0, sizeof(grpc_slice)); + if (len <= sizeof(slice.data.inlined.bytes)) { + slice.refcount = nullptr; + slice.data.inlined.length = len; +@@ -206,7 +207,7 @@ grpc_slice grpc_slice_from_moved_string(grpc_core::UniquePtr p) { + } + + grpc_slice grpc_slice_from_cpp_string(std::string str) { +- grpc_slice slice; ++ grpc_slice slice = { 0 }; + if (str.size() <= sizeof(slice.data.inlined.bytes)) { + slice.refcount = nullptr; + slice.data.inlined.length = str.size(); +diff --git a/src/core/lib/surface/server.cc b/src/core/lib/surface/server.cc +index 141b16e345..89d9d6dafd 100644 +--- a/src/core/lib/surface/server.cc ++++ b/src/core/lib/surface/server.cc +@@ -902,7 +902,7 @@ grpc_call_error Server::QueueRequestedCall(size_t cq_idx, RequestedCall* rc) { + FailCall(cq_idx, rc, GRPC_ERROR_CREATE("Server Shutdown")); + return GRPC_CALL_OK; + } +- RequestMatcherInterface* rm; ++ RequestMatcherInterface* rm = nullptr; + switch (rc->type) { + case RequestedCall::Type::BATCH_CALL: + rm = unregistered_request_matcher_.get(); diff --git a/dev/vcpkg/ports/grpc/00009-use-system-upb.patch b/dev/vcpkg/ports/grpc/00009-use-system-upb.patch new file mode 100644 index 000000000000..8bafe8452b2e --- /dev/null +++ b/dev/vcpkg/ports/grpc/00009-use-system-upb.patch @@ -0,0 +1,91 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 23098aa578..a8e8bc274b 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -85,6 +85,9 @@ set_property(CACHE gRPC_SSL_PROVIDER PROPERTY STRINGS "module" "package") + set(gRPC_PROTOBUF_PROVIDER "module" CACHE STRING "Provider of protobuf library") + set_property(CACHE gRPC_PROTOBUF_PROVIDER PROPERTY STRINGS "module" "package") + ++set(gRPC_UPB_PROVIDER "module" CACHE STRING "Provider of upb library") ++set_property(CACHE gRPC_UPB_PROVIDER PROPERTY STRINGS "module" "package") ++ + set(gRPC_PROTOBUF_PACKAGE_TYPE "" CACHE STRING "Algorithm for searching protobuf package") + set_property(CACHE gRPC_PROTOBUF_PACKAGE_TYPE PROPERTY STRINGS "CONFIG" "MODULE") + +@@ -1631,6 +1634,7 @@ target_link_libraries(gpr + absl::time + absl::optional + absl::variant ++ ${_gRPC_UPB_LIBRARIES} + ) + if(_gRPC_PLATFORM_ANDROID) + target_link_libraries(gpr +@@ -2435,7 +2439,6 @@ target_link_libraries(grpc + gpr + ${_gRPC_SSL_LIBRARIES} + address_sorting +- upb + ) + if(_gRPC_PLATFORM_IOS OR _gRPC_PLATFORM_MAC) + target_link_libraries(grpc "-framework CoreFoundation") +@@ -2979,7 +2982,6 @@ target_link_libraries(grpc_unsecure + absl::utility + gpr + address_sorting +- upb + ) + if(_gRPC_PLATFORM_IOS OR _gRPC_PLATFORM_MAC) + target_link_libraries(grpc_unsecure "-framework CoreFoundation") +@@ -4251,6 +4253,7 @@ endif() + + endif() + ++if (gRPC_UPB_PROVIDER STREQUAL "module") + add_library(upb + third_party/upb/third_party/utf8_range/naive.c + third_party/upb/third_party/utf8_range/range2-neon.c +@@ -4319,7 +4322,7 @@ if(gRPC_INSTALL) + ARCHIVE DESTINATION ${gRPC_INSTALL_LIBDIR} + ) + endif() +- ++endif() + + if(gRPC_BUILD_TESTS) + +diff --git a/cmake/gRPCConfig.cmake.in b/cmake/gRPCConfig.cmake.in +index 3623f4aa5e..df6ced560e 100644 +--- a/cmake/gRPCConfig.cmake.in ++++ b/cmake/gRPCConfig.cmake.in +@@ -8,6 +8,7 @@ list(APPEND CMAKE_MODULE_PATH ${CMAKE_CURRENT_LIST_DIR}/modules) + @_gRPC_FIND_CARES@ + @_gRPC_FIND_ABSL@ + @_gRPC_FIND_RE2@ ++@_gRPC_FIND_UPB@ + + # Targets + include(${CMAKE_CURRENT_LIST_DIR}/gRPCTargets.cmake) +diff --git a/cmake/upb.cmake b/cmake/upb.cmake +index f2a0e508c3..09751f5ef0 100644 +--- a/cmake/upb.cmake ++++ b/cmake/upb.cmake +@@ -12,9 +12,19 @@ + # See the License for the specific language governing permissions and + # limitations under the License. + ++set(_gRPC_UPB_GRPC_GENERATED_DIR "${CMAKE_CURRENT_SOURCE_DIR}/src/core/ext/upb-generated" "${CMAKE_CURRENT_SOURCE_DIR}/src/core/ext/upbdefs-generated") ++if (gRPC_UPB_PROVIDER STREQUAL "module") ++ + set(UPB_ROOT_DIR ${CMAKE_CURRENT_SOURCE_DIR}/third_party/upb) + + set(_gRPC_UPB_INCLUDE_DIR "${UPB_ROOT_DIR}") + set(_gRPC_UPB_GRPC_GENERATED_DIR "${CMAKE_CURRENT_SOURCE_DIR}/src/core/ext/upb-generated" "${CMAKE_CURRENT_SOURCE_DIR}/src/core/ext/upbdefs-generated") + + set(_gRPC_UPB_LIBRARIES upb) ++ ++elseif(gRPC_UPB_PROVIDER STREQUAL "package") ++ find_package(upb CONFIG REQUIRED) ++ set(_gRPC_UPB_LIBRARIES upb::fastdecode upb::json upb::upb upb::utf8_range upb::textformat upb::reflection upb::descriptor_upb_proto) ++ set(_gRPC_UPB_INCLUDE_DIR) ++ set(_gRPC_FIND_UPB "if(NOT upb_FOUND)\n find_package(upb CONFIG REQUIRED)\nendif()") ++endif() diff --git a/dev/vcpkg/ports/grpc/00012-fix-use-cxx17.patch b/dev/vcpkg/ports/grpc/00012-fix-use-cxx17.patch new file mode 100644 index 000000000000..56bc4367608e --- /dev/null +++ b/dev/vcpkg/ports/grpc/00012-fix-use-cxx17.patch @@ -0,0 +1,16 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 25990a5d8a..ba8df92858 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -326,6 +326,11 @@ include(cmake/xxhash.cmake) + include(cmake/zlib.cmake) + include(cmake/download_archive.cmake) + ++if (ABSL_USE_CXX17) ++ message(STATUS "Found absl uses CXX17, enable CXX17 feature.") ++ set(CMAKE_CXX_STANDARD 17) ++endif() ++ + # Setup external proto library at third_party/envoy-api with 2 download URLs + if (NOT EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/third_party/envoy-api) + # Download the archive via HTTP, validate the checksum, and extract to third_party/envoy-api. diff --git a/dev/vcpkg/ports/grpc/00014-pkgconfig-upbdefs.patch b/dev/vcpkg/ports/grpc/00014-pkgconfig-upbdefs.patch new file mode 100644 index 000000000000..1c2bda5b1b94 --- /dev/null +++ b/dev/vcpkg/ports/grpc/00014-pkgconfig-upbdefs.patch @@ -0,0 +1,13 @@ +diff --git a/CMakeLists.txt b/CMakeLists.txt +index 48019cce95..1eda700ae8 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -23186,7 +23186,7 @@ generate_pkgconfig( + "high performance general RPC framework" + "${gRPC_CORE_VERSION}" + "gpr openssl absl_any_invocable absl_base absl_bind_front absl_cleanup absl_cord absl_core_headers absl_flat_hash_map absl_flat_hash_set absl_function_ref absl_hash absl_inlined_vector absl_memory absl_optional absl_random_random absl_span absl_status absl_statusor absl_str_format absl_strings absl_synchronization absl_time absl_type_traits absl_utility absl_variant" +- "-lgrpc -laddress_sorting -lre2 -lupb -lcares -lz" ++ "-lgrpc -laddress_sorting -lre2 -lupb_textformat -lupb_mini_table -lupb -lupb_collections -lupb_reflection -lupb_extension_registry -lupb_json -lupb_fastdecode -lupb_utf8_range -ldescriptor_upb_proto -lcares -lz" + "" + "grpc.pc") + diff --git a/dev/vcpkg/ports/grpc/00015-disable-download-archive.patch b/dev/vcpkg/ports/grpc/00015-disable-download-archive.patch new file mode 100644 index 000000000000..b28bc72a7c91 --- /dev/null +++ b/dev/vcpkg/ports/grpc/00015-disable-download-archive.patch @@ -0,0 +1,12 @@ +diff --git a/cmake/download_archive.cmake b/cmake/download_archive.cmake +index 820aafafb7..a59b785c7e 100644 +--- a/cmake/download_archive.cmake ++++ b/cmake/download_archive.cmake +@@ -19,6 +19,7 @@ file(MAKE_DIRECTORY ${_download_archive_TEMPORARY_DIR}) + # Note that strip_prefix strips the directory path prefix of the extracted + # archive content, and it may strip multiple directories. + function(download_archive destination url hash strip_prefix) ++ return() + # Fetch and validate + set(_TEMPORARY_FILE ${_download_archive_TEMPORARY_DIR}/${strip_prefix}.tar.gz) + message(STATUS "Downloading from ${url}, if failed, please try configuring again") diff --git a/dev/vcpkg/ports/grpc/gRPCTargets-vcpkg-tools.cmake b/dev/vcpkg/ports/grpc/gRPCTargets-vcpkg-tools.cmake new file mode 100644 index 000000000000..1ed3509c9408 --- /dev/null +++ b/dev/vcpkg/ports/grpc/gRPCTargets-vcpkg-tools.cmake @@ -0,0 +1,10 @@ +file(GLOB GRPC_PLUGINS "${_IMPORT_PREFIX}/../@HOST_TRIPLET@/tools/grpc/grpc_*_plugin*") + +foreach(PLUGIN ${GRPC_PLUGINS}) + get_filename_component(PLUGIN_NAME "${PLUGIN}" NAME_WE) + add_executable(gRPC::${PLUGIN_NAME} IMPORTED) + set_property(TARGET gRPC::${PLUGIN_NAME} APPEND PROPERTY IMPORTED_CONFIGURATIONS RELEASE) + set_target_properties(gRPC::${PLUGIN_NAME} PROPERTIES + IMPORTED_LOCATION_RELEASE "${PLUGIN}" + ) +endforeach() diff --git a/dev/vcpkg/ports/grpc/grpc_gcc9.patch b/dev/vcpkg/ports/grpc/grpc_gcc9.patch new file mode 100644 index 000000000000..b64cbc1789e7 --- /dev/null +++ b/dev/vcpkg/ports/grpc/grpc_gcc9.patch @@ -0,0 +1,13 @@ +diff --git a/src/core/ext/xds/xds_listener.h b/src/core/ext/xds/xds_listener.h +index 61427483f1..a3bfed0780 100644 +--- a/src/core/ext/xds/xds_listener.h ++++ b/src/core/ext/xds/xds_listener.h +@@ -79,6 +79,8 @@ struct XdsListenerResource : public XdsResourceType::ResourceData { + }; + + struct DownstreamTlsContext { ++ DownstreamTlsContext() {} ++ + CommonTlsContext common_tls_context; + bool require_client_certificate = false; + diff --git a/dev/vcpkg/ports/grpc/portfile.cmake b/dev/vcpkg/ports/grpc/portfile.cmake new file mode 100644 index 000000000000..a88f371d6b13 --- /dev/null +++ b/dev/vcpkg/ports/grpc/portfile.cmake @@ -0,0 +1,99 @@ +if(VCPKG_TARGET_IS_WINDOWS) + vcpkg_check_linkage(ONLY_STATIC_LIBRARY) +endif() + +vcpkg_from_github( + OUT_SOURCE_PATH SOURCE_PATH + REPO grpc/grpc + REF v1.51.1 + SHA512 1bc8e7a5a15b2dca88527d111cde398b0dc1921bbc945c6df8225b4225b8ac0b43155bcf743230ce7b5962d1ab948e9363229c98a879b1befc7a939a290fb888 + HEAD_REF master + PATCHES + 00001-fix-uwp.patch + 00002-static-linking-in-linux.patch + 00003-undef-base64-macro.patch + 00004-link-gdi32-on-windows.patch + 00005-fix-uwp-error.patch + 00009-use-system-upb.patch + snprintf.patch + 00012-fix-use-cxx17.patch + 00014-pkgconfig-upbdefs.patch + 00015-disable-download-archive.patch + grpc_gcc9.patch +) + +if(NOT TARGET_TRIPLET STREQUAL HOST_TRIPLET) + vcpkg_add_to_path(PREPEND "${CURRENT_HOST_INSTALLED_DIR}/tools/grpc") +endif() + +string(COMPARE EQUAL "${VCPKG_CRT_LINKAGE}" "static" gRPC_MSVC_STATIC_RUNTIME) +string(COMPARE EQUAL "${VCPKG_LIBRARY_LINKAGE}" "static" gRPC_STATIC_LINKING) + +if(VCPKG_TARGET_IS_UWP) + set(cares_CARES_PROVIDER OFF) +else() + set(cares_CARES_PROVIDER "package") +endif() + +vcpkg_check_features( + OUT_FEATURE_OPTIONS FEATURE_OPTIONS + FEATURES + codegen gRPC_BUILD_CODEGEN +) + +vcpkg_cmake_configure( + SOURCE_PATH "${SOURCE_PATH}" + OPTIONS ${FEATURE_OPTIONS} + -DgRPC_INSTALL=ON + -DgRPC_BUILD_TESTS=OFF + -DgRPC_STATIC_LINKING=${gRPC_STATIC_LINKING} + -DgRPC_MSVC_STATIC_RUNTIME=${gRPC_MSVC_STATIC_RUNTIME} + -DgRPC_ZLIB_PROVIDER=package + -DgRPC_SSL_PROVIDER=package + -DgRPC_PROTOBUF_PROVIDER=package + -DgRPC_ABSL_PROVIDER=package + -DgRPC_UPB_PROVIDER=package + -DgRPC_RE2_PROVIDER=package + -DgRPC_PROTOBUF_PACKAGE_TYPE=CONFIG + -DgRPC_CARES_PROVIDER=${cares_CARES_PROVIDER} + -DgRPC_BENCHMARK_PROVIDER=none + -DgRPC_INSTALL_BINDIR:STRING=bin + -DgRPC_INSTALL_LIBDIR:STRING=lib + -DgRPC_INSTALL_INCLUDEDIR:STRING=include + -DgRPC_INSTALL_CMAKEDIR:STRING=share/grpc + "-D_gRPC_PROTOBUF_PROTOC_EXECUTABLE=${CURRENT_HOST_INSTALLED_DIR}/tools/protobuf/protoc${VCPKG_HOST_EXECUTABLE_SUFFIX}" + "-DProtobuf_PROTOC_EXECUTABLE=${CURRENT_HOST_INSTALLED_DIR}/tools/protobuf/protoc${VCPKG_HOST_EXECUTABLE_SUFFIX}" + MAYBE_UNUSED_VARIABLES + gRPC_MSVC_STATIC_RUNTIME +) + +vcpkg_cmake_install(ADD_BIN_TO_PATH) + +vcpkg_cmake_config_fixup() + +if (gRPC_BUILD_CODEGEN) + vcpkg_copy_tools( + AUTO_CLEAN + TOOL_NAMES + grpc_php_plugin + grpc_python_plugin + grpc_node_plugin + grpc_objective_c_plugin + grpc_csharp_plugin + grpc_cpp_plugin + grpc_ruby_plugin + ) +else() + configure_file("${CMAKE_CURRENT_LIST_DIR}/gRPCTargets-vcpkg-tools.cmake" "${CURRENT_PACKAGES_DIR}/share/grpc/gRPCTargets-vcpkg-tools.cmake" @ONLY) +endif() + +file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/debug/share" "${CURRENT_PACKAGES_DIR}/debug/include") + +vcpkg_copy_pdbs() +if (VCPKG_TARGET_IS_WINDOWS) + file(REMOVE_RECURSE "${CURRENT_PACKAGES_DIR}/lib/pkgconfig" "${CURRENT_PACKAGES_DIR}/debug/lib/pkgconfig") +else() + vcpkg_fixup_pkgconfig() +endif() + +file(INSTALL "${SOURCE_PATH}/LICENSE" DESTINATION "${CURRENT_PACKAGES_DIR}/share/${PORT}" RENAME copyright) diff --git a/dev/vcpkg/ports/grpc/snprintf.patch b/dev/vcpkg/ports/grpc/snprintf.patch new file mode 100644 index 000000000000..b5012ad2dba8 --- /dev/null +++ b/dev/vcpkg/ports/grpc/snprintf.patch @@ -0,0 +1,12 @@ +diff --git a/src/core/tsi/alts/crypt/aes_gcm.cc b/src/core/tsi/alts/crypt/aes_gcm.cc +index b761224..88a3d6b 100644 +--- a/src/core/tsi/alts/crypt/aes_gcm.cc ++++ b/src/core/tsi/alts/crypt/aes_gcm.cc +@@ -19,6 +19,7 @@ + #include + + #include ++#include + + #include + #include diff --git a/dev/vcpkg/ports/grpc/vcpkg-cmake-wrapper.cmake b/dev/vcpkg/ports/grpc/vcpkg-cmake-wrapper.cmake new file mode 100644 index 000000000000..9adf0288084d --- /dev/null +++ b/dev/vcpkg/ports/grpc/vcpkg-cmake-wrapper.cmake @@ -0,0 +1,2 @@ +list(REMOVE_AT ARGS 0) +_find_package(gRPC ${ARGS}) # Shouldn't this be fixed downstream instead of using a Wrapper? \ No newline at end of file diff --git a/dev/vcpkg/ports/grpc/vcpkg.json b/dev/vcpkg/ports/grpc/vcpkg.json new file mode 100644 index 000000000000..35dd58509882 --- /dev/null +++ b/dev/vcpkg/ports/grpc/vcpkg.json @@ -0,0 +1,48 @@ +{ + "name": "grpc", + "version-semver": "1.51.1", + "port-version": 1, + "description": "An RPC library and framework", + "homepage": "https://github.com/grpc/grpc", + "license": "Apache-2.0", + "dependencies": [ + "abseil", + { + "name": "c-ares", + "platform": "!uwp" + }, + { + "name": "grpc", + "host": true, + "features": [ + "codegen" + ] + }, + "openssl", + "protobuf", + { + "name": "protobuf", + "host": true + }, + "re2", + "upb", + { + "name": "vcpkg-cmake", + "host": true + }, + { + "name": "vcpkg-cmake-config", + "host": true + }, + "zlib" + ], + "features": { + "absl-sync": { + "description": "Deprecated." + }, + "codegen": { + "description": "Build code generator machinery", + "supports": "!uwp" + } + } +} diff --git a/dev/vcpkg/triplets/x64-linux-avx.cmake b/dev/vcpkg/triplets/x64-linux-avx.cmake index ece4a74bafd3..a80838f25868 100644 --- a/dev/vcpkg/triplets/x64-linux-avx.cmake +++ b/dev/vcpkg/triplets/x64-linux-avx.cmake @@ -1,3 +1,4 @@ +set(VCPKG_BUILD_TYPE release) set(VCPKG_TARGET_ARCHITECTURE x64) set(VCPKG_CRT_LINKAGE dynamic) set(VCPKG_LIBRARY_LINKAGE static) diff --git a/dev/vcpkg/vcpkg.json b/dev/vcpkg/vcpkg.json index 2b8fc392a004..09feb4041618 100644 --- a/dev/vcpkg/vcpkg.json +++ b/dev/vcpkg/vcpkg.json @@ -2,7 +2,7 @@ "$schema": "https://raw.githubusercontent.com/microsoft/vcpkg-tool/main/docs/vcpkg.schema.json", "builtin-baseline": "a7b6122f6b6504d16d96117336a0562693579933", "dependencies": ["jemalloc"], - "default-features": ["arrow", "velox", "velox-s3", "velox-hdfs"], + "default-features": ["arrow", "velox", "velox-s3", "velox-gcs", "velox-hdfs"], "features": { "arrow": { "description": "Apache Arrow", @@ -83,6 +83,17 @@ } ] }, + "velox-gcs": { + "description": "Velox GCS Support", + "dependencies": [ + { + "name": "google-cloud-cpp", + "features": [ + "storage" + ] + } + ] + }, "velox-hdfs": { "description": "Velox HDFS Support", "dependencies": [ diff --git a/docs/Configuration.md b/docs/Configuration.md index 49c8c651263a..e66c5e6034e7 100644 --- a/docs/Configuration.md +++ b/docs/Configuration.md @@ -39,7 +39,8 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.sql.columnar.shuffle.codec | Set up the codec to be used for Columnar Shuffle. If this configuration is not set, will check the value of spark.io.compression.codec. By default, Gluten use software compression. Valid options for software compression are lz4, zstd. Valid options for QAT and IAA is gzip. | lz4 | | spark.gluten.sql.columnar.shuffle.codecBackend | Enable using hardware accelerators for shuffle de/compression. Valid options are QAT and IAA. | | | spark.gluten.sql.columnar.shuffle.compressionMode | Setting different compression mode in shuffle, Valid options are buffer and rowvector, buffer option compress each buffer of RowVector individually into one pre-allocated large buffer, rowvector option first copies each buffer of RowVector to a large buffer and then compress the entire buffer in one go. | buffer | -| 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.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.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.native.bloomFilter | Enable or Disable native runtime bloom filter. | true | @@ -50,6 +51,9 @@ You can add these configurations into spark-defaults.conf to enable or disable t | spark.gluten.loadLibFromJar | Controls whether to load dynamic link library from a packed jar for gluten/cpp. Not applicable to static build and clickhouse backend. | false | | spark.gluten.sql.columnar.force.hashagg | Force to use hash agg to replace sort agg. | 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.columnar.backend.velox.bloomFilter.expectedNumItems | The default number of expected items for the velox bloomfilter. | 1000000L | +| spark.gluten.sql.columnar.backend.velox.bloomFilter.numBits | The default number of bits to use for the velox bloom filter. | 8388608L | +| spark.gluten.sql.columnar.backend.velox.bloomFilter.maxNumBits | The max number of bits to use for the velox bloom filter. | 4194304L | Below is an example for spark-default.conf, if you are using conda to install OAP project. diff --git a/docs/developers/HowTo.md b/docs/developers/HowTo.md index bc81c5deeca1..e4bab4a26883 100644 --- a/docs/developers/HowTo.md +++ b/docs/developers/HowTo.md @@ -41,32 +41,14 @@ To debug C++, you have to generate the example files, the example files consist You can generate the example files by the following steps: -1. get and build Arrow +1. build Velox and Gluten CPP ``` -cd gluten_home/ep/build-arrow/src -./get_arrow.sh -./build_arrow.sh -``` - -2. get and build Velox -``` -cd gluten_home/ep/build-velox/src -./get_velox.sh -./build_velox.sh --build_type=Debug -``` - -3. compile the CPP -``` -cd gluten_home/cpp -mkdir build -cd build -cmake -DBUILD_VELOX_BACKEND=ON -DBUILD_TESTS=ON -DBUILD_BENCHMARKS=ON -DCMAKE_BUILD_TYPE=Debug .. -make -j +gluten_home/dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --build_type=Debug ``` - Compiling with `--build_type=Debug` is good for debugging. - The executable file `generic_benchmark` will be generated under the directory of `gluten_home/cpp/build/velox/benchmarks/`. -4. build Gluten and generate the example files +2. build Gluten and generate the example files ``` cd gluten_home mvn clean package -Pspark-3.2 -Pbackends-velox -Prss @@ -87,7 +69,7 @@ gluten_home/backends-velox/generated-native-benchmark/ └── _SUCCESS ``` -5. now, run benchmarks with GDB +3. now, run benchmarks with GDB ``` cd gluten_home/cpp/build/velox/benchmarks/ gdb generic_benchmark @@ -99,15 +81,15 @@ gdb generic_benchmark - Actually, you can debug `generic_benchmark` with any gdb commands as debugging normal C++ program, because the `generic_benchmark` is a pure C++ executable file in fact. -6. `gdb-tui` is a valuable feature and is worth trying. You can get more help from the online docs. +4. `gdb-tui` is a valuable feature and is worth trying. You can get more help from the online docs. [gdb-tui](https://sourceware.org/gdb/onlinedocs/gdb/TUI.html) -7. you can start `generic_benchmark` with specific JSON plan and input files +5. you can start `generic_benchmark` with specific JSON plan and input files - If you omit them, the `example.json, example_lineitem + example_orders` under the directory of `gluten_home/backends-velox/generated-native-benchmark` will be used as default. - You can also edit the file `example.json` to custom the Substrait plan or specify the inputs files placed in the other directory. -8. get more detail information about benchmarks from [MicroBenchmarks](./MicroBenchmarks.md) +6. get more detail information about benchmarks from [MicroBenchmarks](./MicroBenchmarks.md) ## 2 How to debug Java/Scala wait to add diff --git a/docs/developers/NewToGluten.md b/docs/developers/NewToGluten.md index 64e5441d18ac..38e88c1195e9 100644 --- a/docs/developers/NewToGluten.md +++ b/docs/developers/NewToGluten.md @@ -58,12 +58,8 @@ You can just refer to [build-gluten-with-velox-backend](../get-started/Velox.md# If you need to debug cpp code, please compile the backend code and gluten cpp code with debug mode. ```bash -## compile velox -./build_velox.sh --build_type=Debug -## compile arrow with tests required library -./build_arrow.sh -## compile gluten cpp with benchmark and tests to debug -cmake -DBUILD_VELOX_BACKEND=ON -DBUILD_TESTS=ON -DBUILD_BENCHMARKS=ON -DCMAKE_BUILD_TYPE=Debug .. +## compile velox backend with benchmark and tests to debug +gluten_home/dev/builddeps-veloxbe.sh --build_tests=ON --build_benchmarks=ON --build_type=Debug ``` If you need to debug the tests in /gluten-ut, You need to compile java code with `-P spark-ut`. diff --git a/docs/developers/UsingGperftoolsInCH.md b/docs/developers/UsingGperftoolsInCH.md new file mode 100644 index 000000000000..f0d5c720b30b --- /dev/null +++ b/docs/developers/UsingGperftoolsInCH.md @@ -0,0 +1,26 @@ +We need using gpertools to find the memory or CPU issue. That's what this document is about. + +## Install gperftools +Install gperftools as described in https://github.com/gperftools/gperftools. +We get the library and the command line tools. + +## Compiler libch.so +Disable jemalloc `-DENABLE_JEMALLOC=OFF` in cpp-ch/CMakeLists.txt, and recompile libch.so. + +## Run Gluten with gperftools +For Spark on Yarn, we can change the submit script to run Gluten with gperftools. +Add the following to the submit script: +``` +export tcmalloc_path=/data2/zzb/gperftools-2.10/.libs/libtcmalloc_and_profiler.so # the path to the tcmalloc library +export LD_PRELOAD=$tcmalloc_path,libch.so # load the library in the driver +--files $tcmalloc_path # upload the library to the cluster +--conf spark.executorEnv.LD_PRELOAD=./libtcmalloc_and_profiler.so,libch.so # load the library in the executor +--conf spark.executorEnv.HEAPPROFILE=/tmp/gluten_heap_perf # set the heap profile path, you can change to CPUPROFILE for CPU profiling +``` + +For thrift server on local machine, note using `export LD_PRELOAD="$tcmalloc_path libch.so" # load the library in the driver` to preload dynamic libraries. + +## Analyze the result +We can get the result in the path we set in the previous step. For example, we can get the result in `/tmp/gluten_heap_perf`. We can use the following website to analyze the result: +https://gperftools.github.io/gperftools/heapprofile.html +https://gperftools.github.io/gperftools/cpuprofile.html diff --git a/docs/developers/UsingJemallocWithCH.md b/docs/developers/UsingJemallocWithCH.md new file mode 100644 index 000000000000..626f7522d7c8 --- /dev/null +++ b/docs/developers/UsingJemallocWithCH.md @@ -0,0 +1,40 @@ +We need using jemalloc to find the memory issue. That's what this document is about. + +## Change code of jemalloc +Use libunwind instead of libgcc to get the backtrace which may cause process hangs. +``` +@@ -177,7 +177,8 @@ target_compile_definitions(_jemalloc PRIVATE -DJEMALLOC_PROF=1) + # At the time ClickHouse uses LLVM libunwind which follows libgcc's way of backtracking. + # + # ClickHouse has to provide `unw_backtrace` method by the means of [commit 8e2b31e](https://github.com/ClickHouse/libunwind/commit/8e2b31e766dd502f6df74909e04a7dbdf5182eb1). +-target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) ++#target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBGCC=1) ++target_compile_definitions (_jemalloc PRIVATE -DJEMALLOC_PROF_LIBUNWIND=1) + target_link_libraries (_jemalloc PRIVATE unwind) + ``` +For more infomation, check https://github.com/jemalloc/jemalloc/issues/2282. + +## Get jeprof +Change to the directory where you want to install jemalloc, and run the following commands: +``` +cd $Clickhouse_SOURCE_PATH/contrib/jemalloc && ./autogen.sh && ./configure.sh && make -j8 +``` +Then we get jeprof in the directory `$Clickhouse_SOURCE_PATH/contrib/jemalloc/bin/jeprof`. + +## Compiler libch.so +Ensure to enable jemalloc `-DENABLE_JEMALLOC=ON` in cpp-ch/CMakeLists.txt, and compile libch.so. + +## Run Gluten with jemalloc heap tools +For Yarn or thrift server, you need add the following to the submit script: +``` +export MALLOC_CONF="prof:true,lg_prof_interval:30" # enable jemalloc heap profiling +``` +You can find more options in https://github.com/jemalloc/jemalloc/wiki/Use-Case%3A-Heap-Profiling. + +## Analyze the result +After you get the heap file, you can use the following command to analyze the result: +Check memory diff, so you can find which part of the code consume the memory. +``` +jeprof --svg /usr/lib/jvm/java-8-openjdk-amd64/bin/java --base=jeprof.3717358.1.i1.heap jeprof.3717358.30.i30.heap > diff.svg +``` +You can find more usage about jeprof with `jeprof --help`. \ No newline at end of file diff --git a/docs/developers/docker_centos8.md b/docs/developers/docker_centos8.md index efbe22e18be1..530ef295772c 100755 --- a/docs/developers/docker_centos8.md +++ b/docs/developers/docker_centos8.md @@ -30,7 +30,7 @@ dnf install -y --setopt=install_weak_deps=False ccache gcc-toolset-9 git wget wh source /opt/rh/gcc-toolset-9/enable || exit 1 -yum install -y java-1.8.0-openjdk-devel +yum install -y java-1.8.0-openjdk-devel patch export JAVA_HOME=/usr/lib/jvm/java-1.8.0-openjdk export PATH=$JAVA_HOME/bin:$PATH diff --git a/docs/get-started/ClickHouse.md b/docs/get-started/ClickHouse.md index 07fba751c15a..30ad1e65be61 100644 --- a/docs/get-started/ClickHouse.md +++ b/docs/get-started/ClickHouse.md @@ -453,7 +453,7 @@ $spark_cmd \ --conf spark.gluten.sql.columnar.loadarrow=false \ --conf spark.gluten.sql.columnar.hashagg.enablefinal=true \ --conf spark.gluten.sql.enable.native.validation=false \ - --conf spark.gluten.sql.columnar.forceshuffledhashjoin=true \ + --conf spark.gluten.sql.columnar.forceShuffledHashJoin=true \ --conf spark.gluten.sql.columnar.backend.ch.runtime_config.hdfs.libhdfs3_conf=$hdfs_conf \ --conf spark.gluten.sql.columnar.backend.ch.runtime_config.logger.level=debug \ --conf spark.plugins=io.glutenproject.GlutenPlugin \ diff --git a/docs/get-started/GlutenUsage.md b/docs/get-started/GlutenUsage.md index 46247c6e9c3b..2b78d2b84530 100644 --- a/docs/get-started/GlutenUsage.md +++ b/docs/get-started/GlutenUsage.md @@ -8,37 +8,21 @@ parent: Getting-Started ## Parameters for buildbundle-veloxbe.sh or builddeps-veloxbe.sh Please set them via `--`, e.g. `--build_type=Release`. -| Parameters | Description | Default value | -|------------------|-------------------------------------------------------------|---------------| -| build_type | build type for arrow, velox & gluten cpp, CMAKE_BUILD_TYPE | Release | -| build_tests | build test code in cpp folder and arrow | OFF | -| build_benchmarks | build benchmark code in cpp folder and arrow | OFF | -| build_jemalloc | build with jemalloc | ON | -| build_protobuf | build protobuf lib | ON | -| 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 | -| enable_s3 | build with s3 lib | OFF | -| enable_hdfs | build with hdfs lib | OFF | -| enable_ep_cache | enable caching for external project build (Arrow and Velox) | OFF | -| skip_build_ep | skip the build of external projects (arrow, velox) | OFF | -| enable_vcpkg | enable vcpkg for static build | OFF | - -## Parameters for get_arrow.sh -Please set them via `--`, e.g., `--enable_custom_codec=ON`. - -| Parameters | Description | Default value | -| ---------- | ----------- | ------------- | -| enable_custom_codec | Apply patch to plugin custom codec (used by QAT/IAA) in Arrow cpp IPC module. | OFF | - -## Parameters for build_arrow.sh -Please set them via `--`, e.g., `--arrow_home=/YOUR/PATH`. - -| Parameters | Description | Default value | -| ---------- | ----------- | ------------- | -| arrow_home | Arrow build path | GLUTEN_DIR/ep/build-arrow/build| -| build_type | ARROW build type, CMAKE_BUILD_TYPE | Release| -| build_tests | Build arrow with -DARROW_JSON=ON | OFF | +| Parameters | Description | Default value | +|------------------|-----------------------------------------------------|---------------| +| build_type | build type for velox & gluten cpp, CMAKE_BUILD_TYPE | Release | +| build_tests | build test code in cpp folder | OFF | +| build_benchmarks | build benchmark code in cpp folder | OFF | +| build_jemalloc | build with jemalloc | ON | +| build_protobuf | build protobuf lib | ON | +| 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 | +| enable_s3 | build with s3 lib | OFF | +| enable_hdfs | build with hdfs lib | OFF | +| enable_ep_cache | enable caching for external project build (Velox) | OFF | +| skip_build_ep | skip the build of external projects (velox) | OFF | +| enable_vcpkg | enable vcpkg for static build | OFF | ## Parameters for build_velox.sh Please set them via `--`, e.g., `--velox_home=/YOUR/PATH`. diff --git a/docs/get-started/Velox.md b/docs/get-started/Velox.md index ffe4e63dd784..acfec93fad37 100644 --- a/docs/get-started/Velox.md +++ b/docs/get-started/Velox.md @@ -4,7 +4,7 @@ title: Gluten with Velox Backend nav_order: 1 parent: Getting-Started --- -Currently, the mvn script can automatically fetch and build all dependency libraries incluing Velox and Arrow. Our nightly build still use Velox under oap-project. +Currently, the mvn script can automatically fetch and build all dependency libraries incluing Velox. Our nightly build still use Velox under oap-project. # Prerequisite @@ -13,14 +13,7 @@ common OS and conda env deployment. Gluten builds with Spark3.2.x and Spark3.3.x now but only fully tested in CI with 3.2.2 and 3.3.1. We will add/update supported/tested versions according to the upstream changes. -Velox uses the script `scripts/setup-xxx.sh` to install all dependency libraries, but Arrow's dependency libraries are not installed. Velox also requires ninja for compilation. -So we need to install all of them manually. Also, we need to set up the `JAVA_HOME` env. Currently, **java 8** is required and the support for java 11/17 is not ready. - -```bash -## run as root -## install gcc and libraries to build arrow -apt-get update && apt-get 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 -``` +we need to set up the `JAVA_HOME` env. Currently, **java 8** is required and the support for java 11/17 is not ready. **For x86_64** @@ -61,7 +54,7 @@ cd /path/to/gluten ./dev/buildbundle-veloxbe.sh ## After a complete build, if you need to re-build the project and only some gluten code is changed, -## you can use the following command to skip building arrow, velox and protobuf. +## you can use the following command to skip building velox and protobuf. # ./dev/buildbundle-veloxbe.sh --skip_build_ep=ON --build_protobuf=OFF ``` @@ -75,30 +68,23 @@ cd /path/to/gluten ./dev/builddeps-veloxbe.sh ``` -**Build Velox or Arrow separately** +**Build Velox separately** -Scripts under `/path/to/gluten/ep/build-xxx/src` provide `get_xxx.sh` and `build_xxx.sh` to build Velox or Arrow separately, you could use these scripts with custom repo/branch/location. +Scripts under `/path/to/gluten/ep/build-velox/src` provide `get_velox.sh` and `build_velox.sh` to build Velox separately, you could use these scripts with custom repo/branch/location. -Velox can use pre-build arrow/parquet lib from ARROW_HOME parsed by --arrow_home to decrease build time. -Gluten cpp module need a required VELOX_HOME parsed by --velox_home and an optional ARROW_HOME by --arrow_home, if you specify custom ep location, make sure these variables be passed correctly. +Velox provides arrow/parquet lib. Gluten cpp module need a required VELOX_HOME parsed by --velox_home, if you specify custom ep location, make sure these variables be passed correctly. ```bash -## fetch Arrow and compile -cd /path/to/gluten/ep/build-arrow/src/ -## you could use custom ep location by --arrow_home=custom_path, make sure specify --arrow_home in build_arrow.sh too. -./get_arrow.sh -./build_arrow.sh - ## fetch Velox and compile cd /path/to/gluten/ep/build-velox/src/ ## you could use custom ep location by --velox_home=custom_path, make sure specify --velox_home in build_velox.sh too. ./get_velox.sh -## make sure specify --arrow_home or --velox_home if you have specified it in get_xxx.sh. +## make sure specify --velox_home if you have specified it in get_velox.sh. ./build_velox.sh ## compile Gluten cpp module cd /path/to/gluten/cpp -## if you use custom velox_home or arrow_home, make sure specified here by --arrow_home or --velox_home +## if you use custom velox_home, make sure specified here by --velox_home ./compile.sh --build_velox_backend=ON ## compile Gluten java module and create package jar @@ -254,6 +240,26 @@ spark.celeborn.storage.hdfs.dir hdfs:///celeborn spark.dynamicAllocation.enabled false ``` +## DeltaLake Support + +Gluten with velox backend supports [DeltaLake](https://delta.io/) table. + +### How to use + +First of all, compile gluten-delta module by a `delta` profile, as follows: + +``` +mvn clean package -Pbackends-velox -Pspark-3.3 -Pdelta -DskipTests +``` + +Then, put the additional gluten-delta jar to the class path (usually it's `$SPARK_HOME/jars`). +The gluten-delta jar is in `gluten-delta/target` directory. + +After the two steps, you can query delta table by gluten/velox without scan's fallback. + +Gluten with velox backends also support the column mapping of delta tables. +About column mapping, see more [here](https://docs.delta.io/latest/delta-column-mapping.html). + # Coverage Spark3.3 has 387 functions in total. ~240 are commonly used. Velox's functions have two category, Presto and Spark. Presto has 124 functions implemented. Spark has 62 functions. Spark functions are verified to have the same result as Vanilla Spark. Some Presto functions have the same result as Vanilla Spark but some others have different. Gluten prefer to use Spark functions firstly. If it's not in Spark's list but implemented in Presto, we currently offload to Presto one until we noted some result mismatch, then we need to reimplement the function in Spark category. Gluten currently offloads 94 functions and 14 operators, more details refer to [Velox Backend's Supported Operators & Functions](../velox-backend-support-progress.md). @@ -693,20 +699,19 @@ All TPC-H and TPC-DS queries are supported in Gluten Velox backend. ## Data preparation -The data generation scripts are [TPC-H dategen script](../backends-velox/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh) and -[TPC-DS dategen script](../backends-velox/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh). +The data generation scripts are [TPC-H dategen script](../../tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh) and +[TPC-DS dategen script](../../tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh). -The used TPC-H and TPC-DS queries are the original ones, and can be accessed from [TPC-DS queries](../gluten-core/src/test/resources/tpcds-queries/tpcds.queries.original) -and [TPC-H queries](../gluten-core/src/test/resources/tpch-queries). +The used TPC-H and TPC-DS queries are the original ones, and can be accessed from [TPC-DS queries](../../gluten-core/src/test/resources/tpcds-queries/tpcds.queries.original) +and [TPC-H queries](../../gluten-core/src/test/resources/tpch-queries). -Some other versions of TPC-DS and TPC-H queries are also provided, but are **not** recommended for testing, including: -- the modified TPC-H queries with "Date-to-String" conversions: [TPC-H non-date queries](../tools/gluten-it/src/main/resources/tpch-queries-nodate) (outdated). -- the modified TPC-DS queries with "Decimal-to-Double": [TPC-DS non-decimal queries](../gluten-core/src/test/resources/tpcds-queries/tpcds.queries.no-decimal) (outdated). -- the modified TPC-DS queries with "Decimal-to-Double" and "Date-to-String" conversions: [TPC-DS modified queries](../tools/gluten-it/src/main/resources/tpcds-queries-nodecimal-nodate) (outdated). +Some other versions of TPC-DS queries are also provided, but are **not** recommended for testing, including: +- the modified TPC-DS queries with "Decimal-to-Double": [TPC-DS non-decimal queries](../../gluten-core/src/test/resources/tpcds-queries/tpcds.queries.no-decimal) (outdated). ## Submit the Spark SQL job -Submit test script from spark-shell. You can find the scala code to [Run TPC-H](../backends-velox/workload/tpch/run_tpch/tpch_parquet.scala) as an example. Please remember to modify the location of TPC-H files as well as TPC-H queries in backends-velox/workload/tpch/run_tpch/tpch_parquet.scala before you run the testing. +Submit test script from spark-shell. You can find the scala code to [Run TPC-H](../../tools/workload/tpch/run_tpch/tpch_parquet.scala) as an example. Please remember to modify +the location of TPC-H files as well as TPC-H queries before you run the testing. ``` var parquet_file_path = "/PATH/TO/TPCH_PARQUET_PATH" @@ -785,3 +790,95 @@ If you want to disable Gluten UI, add a config when submitting `--conf spark.glu ## History server Gluten UI also supports Spark history server. Add gluten-ui jar into the history server classpath, e.g., $SPARK_HOME/jars, then restart history server. + +## Native plan string + +Gluten supports inject native plan string into Spark explain with formatted mode by setting `--conf spark.gluten.sql.injectNativePlanStringToExplain=true`. +Here is an example, how Gluten show the native plan string. + +``` +(9) WholeStageCodegenTransformer (2) +Input [6]: [c1#0L, c2#1L, c3#2L, c1#3L, c2#4L, c3#5L] +Arguments: false +Native Plan: +-- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT + -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT + -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT +``` + +## Native plan with stats + +Gluten supports print native plan with stats to executor system output stream by setting `--conf spark.gluten.sql.debug=true`. +Note that, the plan string with stats is task level which may cause executor log size big. Here is an example, how Gluten show the native plan string with stats. + +``` +I20231121 10:19:42.348845 90094332 WholeStageResultIterator.cc:220] Native Plan with stats for: [Stage: 1 TID: 16] +-- Project[expressions: (n3_6:BIGINT, "n0_0"), (n3_7:BIGINT, "n0_1"), (n3_8:BIGINT, "n0_2"), (n3_9:BIGINT, "n1_0"), (n3_10:BIGINT, "n1_1"), (n3_11:BIGINT, "n1_2")] -> n3_6:BIGINT, n3_7:BIGINT, n3_8:BIGINT, n3_9:BIGINT, n3_10:BIGINT, n3_11:BIGINT + Output: 27 rows (3.56KB, 3 batches), Cpu time: 10.58us, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 + queuedWallNanos sum: 2.00us, count: 1, min: 2.00us, max: 2.00us + runningAddInputWallNanos sum: 626ns, count: 1, min: 626ns, max: 626ns + runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningGetOutputWallNanos sum: 5.54us, count: 1, min: 5.54us, max: 5.54us + -- HashJoin[INNER n1_1=n0_1] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT, n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + Output: 27 rows (3.56KB, 3 batches), Cpu time: 223.00us, Blocked wall time: 0ns, Peak memory: 93.12KB, Memory allocations: 15 + HashBuild: Input: 10 rows (960B, 10 batches), Output: 0 rows (0B, 0 batches), Cpu time: 185.67us, Blocked wall time: 0ns, Peak memory: 68.00KB, Memory allocations: 2, Threads: 1 + distinctKey0 sum: 4, count: 1, min: 4, max: 4 + hashtable.capacity sum: 4, count: 1, min: 4, max: 4 + hashtable.numDistinct sum: 10, count: 1, min: 10, max: 10 + hashtable.numRehashes sum: 1, count: 1, min: 1, max: 1 + queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + rangeKey0 sum: 4, count: 1, min: 4, max: 4 + runningAddInputWallNanos sum: 1.27ms, count: 1, min: 1.27ms, max: 1.27ms + runningFinishWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningGetOutputWallNanos sum: 1.29us, count: 1, min: 1.29us, max: 1.29us + H23/11/21 10:19:42 INFO TaskSetManager: Finished task 3.0 in stage 1.0 (TID 13) in 335 ms on 10.221.97.35 (executor driver) (1/10) +ashProbe: Input: 9 rows (864B, 3 batches), Output: 27 rows (3.56KB, 3 batches), Cpu time: 37.33us, Blocked wall time: 0ns, Peak memory: 25.12KB, Memory allocations: 13, Threads: 1 + dynamicFiltersProduced sum: 1, count: 1, min: 1, max: 1 + queuedWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningAddInputWallNanos sum: 4.54us, count: 1, min: 4.54us, max: 4.54us + runningFinishWallNanos sum: 83ns, count: 1, min: 83ns, max: 83ns + runningGetOutputWallNanos sum: 29.08us, count: 1, min: 29.08us, max: 29.08us + -- TableScan[table: hive_table, range filters: [(c2, Filter(IsNotNull, deterministic, null not allowed))]] -> n1_0:BIGINT, n1_1:BIGINT, n1_2:BIGINT + Input: 9 rows (864B, 3 batches), Output: 9 rows (864B, 3 batches), Cpu time: 630.75us, Blocked wall time: 0ns, Peak memory: 2.44KB, Memory allocations: 63, Threads: 1, Splits: 3 + dataSourceWallNanos sum: 102.00us, count: 1, min: 102.00us, max: 102.00us + dynamicFiltersAccepted sum: 1, count: 1, min: 1, max: 1 + flattenStringDictionaryValues sum: 0, count: 1, min: 0, max: 0 + ioWaitNanos sum: 312.00us, count: 1, min: 312.00us, max: 312.00us + localReadBytes sum: 0B, count: 1, min: 0B, max: 0B + numLocalRead sum: 0, count: 1, min: 0, max: 0 + numPrefetch sum: 0, count: 1, min: 0, max: 0 + numRamRead sum: 0, count: 1, min: 0, max: 0 + numStorageRead sum: 6, count: 1, min: 6, max: 6 + overreadBytes sum: 0B, count: 1, min: 0B, max: 0B + prefetchBytes sum: 0B, count: 1, min: 0B, max: 0B + queryThreadIoLatency sum: 12, count: 1, min: 12, max: 12 + ramReadBytes sum: 0B, count: 1, min: 0B, max: 0B + runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningFinishWallNanos sum: 125ns, count: 1, min: 125ns, max: 125ns + runningGetOutputWallNanos sum: 1.07ms, count: 1, min: 1.07ms, max: 1.07ms + skippedSplitBytes sum: 0B, count: 1, min: 0B, max: 0B + skippedSplits sum: 0, count: 1, min: 0, max: 0 + skippedStrides sum: 0, count: 1, min: 0, max: 0 + storageReadBytes sum: 3.44KB, count: 1, min: 3.44KB, max: 3.44KB + totalScanTime sum: 0ns, count: 1, min: 0ns, max: 0ns + -- ValueStream[] -> n0_0:BIGINT, n0_1:BIGINT, n0_2:BIGINT + Input: 0 rows (0B, 0 batches), Output: 10 rows (960B, 10 batches), Cpu time: 1.03ms, Blocked wall time: 0ns, Peak memory: 0B, Memory allocations: 0, Threads: 1 + runningAddInputWallNanos sum: 0ns, count: 1, min: 0ns, max: 0ns + runningFinishWallNanos sum: 54.62us, count: 1, min: 54.62us, max: 54.62us + runningGetOutputWallNanos sum: 1.10ms, count: 1, min: 1.10ms, max: 1.10ms +``` + +# Gluten Implicits + +Gluten provides a helper class to get the fallback summary from a Spark Dataset. + +``` +import org.apache.spark.sql.execution.GlutenImplicits._ +val df = spark.sql("SELECT * FROM t") +df.fallbackSummary +``` + +Note that, if AQE is enabled, but the query is not materialized, then it will re-plan +the query execution with disabled AQE. It is a workaround to get the final plan, and it may +cause the inconsistent results with a materialized query. However, we have no choice. diff --git a/docs/velox-backend-limitations.md b/docs/velox-backend-limitations.md index eaea3772d7eb..9a8b1348cbb2 100644 --- a/docs/velox-backend-limitations.md +++ b/docs/velox-backend-limitations.md @@ -86,11 +86,22 @@ Exception occurs when Velox TableScan is used to read files with unsupported com | DWRF | Y | Y | Y | Y | Y | Y | N | -### Parquet Write +### Native Write -#### Offload hive file format to velox (offload) +#### Offload native write to velox -We implemented the insert into command by overriding HiveFileFormat in Vanilla spark. And you need to ensure preferentially load the Gluten jar to overwrite the jar of vanilla spark. Refer to [How to prioritize loading Gluten jars in Spark](https://github.com/oap-project/gluten/blob/main/docs/developers/NewToGluten.md). It should be noted that if the user also modifies the HiveFileFormat, the user's changes may be overwritten. +We implemented write support by overriding the following vanilla Spark classes. And you need to ensure preferentially load the Gluten jar to overwrite the jar of vanilla spark. Refer to [How to prioritize loading Gluten jars in Spark](https://github.com/oap-project/gluten/blob/main/docs/velox-backend-troubleshooting.md#incompatible-class-error-when-using-native-writer). It should be noted that if the user also modifies the following overriding classes, the user's changes may be overwritten. + +``` +./shims/spark32/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +./shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala + +``` ### Velox Parquet Write @@ -108,14 +119,23 @@ Parquet write only support three configs, other will not take effect. - sql conf: `spark.gluten.sql.native.parquet.write.blockRows` - option: `parquet.block.rows` -#### Write a partitioned or bucketed table (exception) +#### Static partition write + +Velox exclusively supports static partition writes and does not support dynamic partition writes. + +```scala +spark.sql("CREATE TABLE t (c int, d long, e long) STORED AS PARQUET partitioned by (c, d)") +spark.sql("INSERT OVERWRITE TABLE t partition(c=1, d=2) SELECT 3 as e") +``` + +#### Write a dynamic partitioned or bucketed table (exception) Velox does not support dynamic partition write and bucket write, e.g., ```scala spark.range(100).selectExpr("id as c1", "id % 7 as p") .write - .format("velox") + .format("parquet") .partitionBy("p") .save(f.getCanonicalPath) ``` @@ -127,7 +147,7 @@ Velox does not create table as select, e.g., ```scala spark.range(100).toDF("id") .write - .format("velox") + .format("parquet") .saveAsTable("velox_ctas") ``` diff --git a/ep/build-arrow/src/build_arrow.sh b/ep/build-arrow/src/build_arrow.sh deleted file mode 100755 index 535144938a10..000000000000 --- a/ep/build-arrow/src/build_arrow.sh +++ /dev/null @@ -1,138 +0,0 @@ -#!/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 -exu - -BUILD_TYPE=release -NPROC=$(nproc --ignore=2) -TARGET_BUILD_COMMIT="" -ARROW_HOME="" -ENABLE_EP_CACHE=OFF - -for arg in "$@"; do - case $arg in - --build_type=*) - BUILD_TYPE=("${arg#*=}") - shift # Remove argument name from processing - ;; - --arrow_home=*) - ARROW_HOME=("${arg#*=}") - shift # Remove argument name from processing - ;; - --enable_ep_cache=*) - ENABLE_EP_CACHE=("${arg#*=}") - shift # Remove argument name from processing - ;; - *) - OTHER_ARGUMENTS+=("$1") - shift # Remove generic argument from processing - ;; - esac -done - -CURRENT_DIR=$( - cd "$(dirname "$BASH_SOURCE")" - pwd -) -if [ "$ARROW_HOME" == "" ]; then - ARROW_HOME="$CURRENT_DIR/../build" -fi - -ARROW_SOURCE_DIR="${ARROW_HOME}/arrow_ep" -ARROW_INSTALL_DIR="${ARROW_HOME}/arrow_install" - -echo "Building Arrow from Source..." -echo "CMAKE Arguments:" -echo "BUILD_TYPE=${BUILD_TYPE}" -echo "ARROW_HOME=${ARROW_HOME}" - -pushd $ARROW_SOURCE_DIR - -TARGET_BUILD_COMMIT=$(git rev-parse --verify HEAD) -if [ -z "$TARGET_BUILD_COMMIT" ]; then - echo "Unable to parse Arrow commit: $TARGET_BUILD_COMMIT." - exit 1 -fi -echo "Target Arrow commit: $TARGET_BUILD_COMMIT" - -if [ $ENABLE_EP_CACHE == "ON" ]; then - if [ -f ${ARROW_HOME}/arrow-commit.cache ]; then - CACHED_BUILT_COMMIT="$(cat ${ARROW_HOME}/arrow-commit.cache)" - if [ -n "$CACHED_BUILT_COMMIT" ]; then - if [ "$TARGET_BUILD_COMMIT" = "$CACHED_BUILT_COMMIT" ]; then - echo "Arrow build of commit $TARGET_BUILD_COMMIT was cached." - exit 0 - else - echo "Found cached commit $CACHED_BUILT_COMMIT for Arrow which is different with target commit $TARGET_BUILD_COMMIT." - fi - fi - fi -else - git clean -dffx :/ - rm -rf $ARROW_INSTALL_DIR - mkdir -p $ARROW_INSTALL_DIR -fi - -if [ -f ${ARROW_HOME}/arrow-commit.cache ]; then - rm -f ${ARROW_HOME}/arrow-commit.cache -fi - -# Arrow CPP libraries -mkdir -p cpp/build -pushd cpp/build -cmake -G Ninja \ - -DARROW_BUILD_STATIC=OFF \ - -DARROW_COMPUTE=ON \ - -DARROW_WITH_RE2=ON \ - -DARROW_FILESYSTEM=ON \ - -DARROW_WITH_LZ4=ON \ - -DARROW_WITH_SNAPPY=ON \ - -DARROW_WITH_ZLIB=ON \ - -DARROW_PARQUET=ON \ - -DARROW_ORC=ON \ - -DARROW_WITH_ZSTD=ON \ - -DARROW_BUILD_SHARED=ON \ - -DARROW_BOOST_USE_SHARED=OFF \ - -DARROW_ZSTD_USE_SHARED=OFF \ - -DARROW_JAVA_JNI_ENABLE_DEFAULT=OFF \ - -DARROW_JEMALLOC=ON \ - -DARROW_SIMD_LEVEL=AVX2 \ - -DARROW_RUNTIME_SIMD_LEVEL=MAX \ - -DARROW_DEPENDENCY_SOURCE=AUTO \ - -DCMAKE_INSTALL_PREFIX=$ARROW_INSTALL_DIR \ - -DCMAKE_INSTALL_LIBDIR=lib \ - -DCMAKE_BUILD_TYPE=$BUILD_TYPE \ - -DCMAKE_EXPORT_COMPILE_COMMANDS=ON \ - .. -cmake --build . --target install -popd - -CPU_TARGET=${CPU_TARGET:-""} -if [ "$CPU_TARGET" == "aarch64" ]; then - # Arrow C Data Interface CPP libraries - pushd java - mvn generate-resources -P generate-libs-cdata-all-os -Darrow.c.jni.dist.dir=$ARROW_INSTALL_DIR -N - popd - - # Arrow Java libraries - pushd java - mvn clean install -P arrow-c-data -pl c -am -DskipTests -Dcheckstyle.skip \ - -Darrow.c.jni.dist.dir=$ARROW_INSTALL_DIR/lib -Dmaven.gitcommitid.skip=true - popd -fi - -echo "Successfully built Arrow from Source." -echo $TARGET_BUILD_COMMIT >"${ARROW_HOME}/arrow-commit.cache" diff --git a/ep/build-arrow/src/get_arrow.sh b/ep/build-arrow/src/get_arrow.sh deleted file mode 100755 index 554d498d20c6..000000000000 --- a/ep/build-arrow/src/get_arrow.sh +++ /dev/null @@ -1,88 +0,0 @@ -#!/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 -exu - -ARROW_REPO=https://github.com/apache/arrow.git -ARROW_BRANCH=apache-arrow-12.0.0 -ARROW_HOME="" - -for arg in "$@"; do - case $arg in - --arrow_repo=*) - ARROW_REPO=("${arg#*=}") - shift # Remove argument name from processing - ;; - --arrow_branch=*) - ARROW_BRANCH=("${arg#*=}") - shift # Remove argument name from processing - ;; - --arrow_home=*) - ARROW_HOME=("${arg#*=}") - shift # Remove argument name from processing - ;; - *) - OTHER_ARGUMENTS+=("$1") - shift # Remove generic argument from processing - ;; - esac -done - -function checkout_code { - TARGET_BUILD_COMMIT="$(git ls-remote $ARROW_REPO $ARROW_BRANCH | awk '{print $1;}')" - if [ -d $ARROW_SOURCE_DIR ]; then - echo "Arrow source folder $ARROW_SOURCE_DIR already exists..." - cd $ARROW_SOURCE_DIR - git init . - EXISTS=$(git show-ref refs/tags/build_$TARGET_BUILD_COMMIT || true) - if [ -z "$EXISTS" ]; then - git fetch $ARROW_REPO $TARGET_BUILD_COMMIT:refs/tags/build_$TARGET_BUILD_COMMIT - fi - git reset --hard HEAD - git checkout refs/tags/build_$TARGET_BUILD_COMMIT - else - git clone $ARROW_REPO -b $ARROW_BRANCH $ARROW_SOURCE_DIR - cd $ARROW_SOURCE_DIR - git checkout $TARGET_BUILD_COMMIT - fi -} - -echo "Preparing Arrow source code..." - -CURRENT_DIR=$( - cd "$(dirname "$BASH_SOURCE")" - pwd -) - -if [ "$ARROW_HOME" == "" ]; then - ARROW_HOME="$CURRENT_DIR/../build" -fi -ARROW_SOURCE_DIR="${ARROW_HOME}/arrow_ep" - -checkout_code - -cat > cpp/src/parquet/symbols.map < + #include + #include + #include diff --git a/ep/build-velox/src/modify_velox.patch b/ep/build-velox/src/modify_velox.patch new file mode 100644 index 000000000000..2410600deb28 --- /dev/null +++ b/ep/build-velox/src/modify_velox.patch @@ -0,0 +1,152 @@ +diff --git a/CMake/Findlz4.cmake b/CMake/Findlz4.cmake +index d49115f12..1aaa8e532 100644 +--- a/CMake/Findlz4.cmake ++++ b/CMake/Findlz4.cmake +@@ -21,18 +21,19 @@ find_package_handle_standard_args(lz4 DEFAULT_MSG LZ4_LIBRARY LZ4_INCLUDE_DIR) + + mark_as_advanced(LZ4_LIBRARY LZ4_INCLUDE_DIR) + +-get_filename_component(liblz4_ext ${LZ4_LIBRARY} EXT) +-if(liblz4_ext STREQUAL ".a") +- set(liblz4_type STATIC) +-else() +- set(liblz4_type SHARED) +-endif() +- + if(NOT TARGET lz4::lz4) +- add_library(lz4::lz4 ${liblz4_type} IMPORTED) +- set_target_properties(lz4::lz4 PROPERTIES INTERFACE_INCLUDE_DIRECTORIES +- "${LZ4_INCLUDE_DIR}") +- set_target_properties( +- lz4::lz4 PROPERTIES IMPORTED_LINK_INTERFACE_LANGUAGES "C" +- IMPORTED_LOCATION "${LZ4_LIBRARIES}") ++ add_library(lz4::lz4 UNKNOWN IMPORTED) ++ set_target_properties(lz4::lz4 PROPERTIES ++ INTERFACE_INCLUDE_DIRECTORIES "${LZ4_INCLUDE_DIR}" ++ IMPORTED_LINK_INTERFACE_LANGUAGES "C" ++ IMPORTED_LOCATION_RELEASE "${LZ4_LIBRARY_RELEASE}") ++ set_property(TARGET lz4::lz4 APPEND PROPERTY ++ IMPORTED_CONFIGURATIONS RELEASE) ++ ++ if(LZ4_LIBRARY_DEBUG) ++ set_property(TARGET lz4::lz4 APPEND PROPERTY ++ IMPORTED_CONFIGURATIONS DEBUG) ++ set_property(TARGET lz4::lz4 PROPERTY ++ IMPORTED_LOCATION_DEBUG "${LZ4_LIBRARY_DEBUG}") ++ endif() + endif() +diff --git a/CMakeLists.txt b/CMakeLists.txt +index b34966b7a..1999a50f0 100644 +--- a/CMakeLists.txt ++++ b/CMakeLists.txt +@@ -211,10 +211,15 @@ if(VELOX_ENABLE_ABFS) + endif() + + if(VELOX_ENABLE_HDFS) +- find_library( +- LIBHDFS3 +- NAMES libhdfs3.so libhdfs3.dylib +- HINTS "${CMAKE_SOURCE_DIR}/hawq/depends/libhdfs3/_build/src/" REQUIRED) ++ find_package(libhdfs3) ++ if(libhdfs3_FOUND AND TARGET HDFS::hdfs3) ++ set(LIBHDFS3 HDFS::hdfs3) ++ else() ++ find_library( ++ LIBHDFS3 ++ NAMES libhdfs3.so libhdfs3.dylib ++ HINTS "${CMAKE_SOURCE_DIR}/hawq/depends/libhdfs3/_build/src/" REQUIRED) ++ endif() + add_definitions(-DVELOX_ENABLE_HDFS3) + endif() + +@@ -380,7 +385,7 @@ resolve_dependency(Boost 1.66.0 COMPONENTS ${BOOST_INCLUDE_LIBRARIES}) + # for reference. find_package(range-v3) + + set_source(gflags) +-resolve_dependency(gflags COMPONENTS shared) ++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/scripts/setup-ubuntu.sh b/scripts/setup-ubuntu.sh +index a1dda4b6c..dc423c826 100755 +--- a/scripts/setup-ubuntu.sh ++++ b/scripts/setup-ubuntu.sh +@@ -31,7 +31,7 @@ export CMAKE_BUILD_TYPE=Release + # Install all velox and folly dependencies. + # The is an issue on 22.04 where a version conflict prevents glog install, + # installing libunwind first fixes this. +-sudo --preserve-env apt update && sudo --preserve-env apt install -y libunwind-dev && \ ++sudo --preserve-env apt update && \ + sudo --preserve-env apt install -y \ + g++ \ + cmake \ +diff --git a/third_party/CMakeLists.txt b/third_party/CMakeLists.txt +index 570af4593..6b20e7908 100644 +--- a/third_party/CMakeLists.txt ++++ b/third_party/CMakeLists.txt +@@ -27,6 +27,7 @@ if(VELOX_ENABLE_ARROW) + set(ARROW_PREFIX "${CMAKE_CURRENT_BINARY_DIR}/arrow_ep") + set(ARROW_CMAKE_ARGS + -DARROW_PARQUET=ON ++ -DARROW_FILESYSTEM=ON + -DARROW_WITH_LZ4=ON + -DARROW_WITH_SNAPPY=ON + -DARROW_WITH_ZLIB=ON +@@ -68,6 +69,7 @@ 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 + URL_HASH ${VELOX_ARROW_BUILD_SHA256_CHECKSUM} + SOURCE_SUBDIR cpp + CMAKE_ARGS ${ARROW_CMAKE_ARGS} +diff --git a/velox/common/process/tests/CMakeLists.txt b/velox/common/process/tests/CMakeLists.txt +index d64466568..52927c4a5 100644 +--- a/velox/common/process/tests/CMakeLists.txt ++++ b/velox/common/process/tests/CMakeLists.txt +@@ -17,4 +17,4 @@ add_executable(velox_process_test TraceContextTest.cpp) + add_test(velox_process_test velox_process_test) + + target_link_libraries(velox_process_test PRIVATE velox_process fmt::fmt gtest +- gtest_main) ++ gtest_main glog::glog gflags::gflags) +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/velox/dwio/common/CMakeLists.txt b/velox/dwio/common/CMakeLists.txt +index c4b767317..8bcf766e7 100644 +--- a/velox/dwio/common/CMakeLists.txt ++++ b/velox/dwio/common/CMakeLists.txt +@@ -73,4 +73,5 @@ target_link_libraries( + velox_exec + Boost::regex + Folly::folly +- glog::glog) ++ glog::glog ++ protobuf::libprotobuf) +diff --git a/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt b/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt +index ce1bc6782..3139314b4 100644 +--- a/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt ++++ b/velox/dwio/parquet/writer/arrow/tests/CMakeLists.txt +@@ -40,7 +40,9 @@ target_link_libraries( + gtest_main + parquet + arrow +- arrow_testing) ++ arrow_testing ++ glog::glog ++ gflags::gflags) + + add_library( + velox_dwio_arrow_parquet_writer_test_lib diff --git a/gluten-celeborn/clickhouse/pom.xml b/gluten-celeborn/clickhouse/pom.xml index 8a5f0337c244..e71dce47d3b6 100755 --- a/gluten-celeborn/clickhouse/pom.xml +++ b/gluten-celeborn/clickhouse/pom.xml @@ -75,9 +75,7 @@
org.apache.spark - spark-hive-thriftserver_${scala.binary.version} - ${spark.version} - test-jar + spark-hive_${scala.binary.version} test diff --git a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornHashBasedColumnarShuffleWriter.scala b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornHashBasedColumnarShuffleWriter.scala index 3cb52dcb4104..61938e263c5d 100644 --- a/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornHashBasedColumnarShuffleWriter.scala +++ b/gluten-celeborn/clickhouse/src/main/scala/org/apache/spark/shuffle/CHCelebornHashBasedColumnarShuffleWriter.scala @@ -17,6 +17,7 @@ package org.apache.spark.shuffle import io.glutenproject.GlutenConfig +import io.glutenproject.backendsapi.clickhouse.CHBackendSettings import io.glutenproject.memory.alloc.CHNativeMemoryAllocators import io.glutenproject.memory.memtarget.MemoryTarget import io.glutenproject.memory.memtarget.Spiller @@ -68,6 +69,7 @@ class CHCelebornHashBasedColumnarShuffleWriter[K, V]( nativeBufferSize, customizedCompressCodec, GlutenConfig.getConf.chColumnarShuffleSpillThreshold, + CHBackendSettings.shuffleHashAlgorithm, celebornPartitionPusher ) CHNativeMemoryAllocators.createSpillable( diff --git a/gluten-celeborn/clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala b/gluten-celeborn/clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala index d6b53a9b9de2..5b283299ece5 100644 --- a/gluten-celeborn/clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala +++ b/gluten-celeborn/clickhouse/src/test/scala/io/glutenproject/execution/GlutenClickHouseRSSColumnarShuffleAQESuite.scala @@ -18,7 +18,7 @@ package io.glutenproject.execution import org.apache.spark.SparkConf import org.apache.spark.sql.execution.CoalescedPartitionSpec -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ColumnarAQEShuffleReadExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec} import org.apache.spark.sql.internal.SQLConf case class TestData(id: Int) @@ -52,7 +52,7 @@ class GlutenClickHouseRSSColumnarShuffleAQESuite assert(df.queryExecution.executedPlan.isInstanceOf[AdaptiveSparkPlanExec]) val colCustomShuffleReaderExecs = collect(df.queryExecution.executedPlan) { - case csr: ColumnarAQEShuffleReadExec => csr + case csr: AQEShuffleReadExec => csr } assert(colCustomShuffleReaderExecs.size == 2) val coalescedPartitionSpec0 = colCustomShuffleReaderExecs(0) diff --git a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala index 7cb93d9e6f5f..6f78a4addb0e 100644 --- a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala +++ b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornHashBasedColumnarShuffleWriter.scala @@ -51,6 +51,8 @@ abstract class CelebornHashBasedColumnarShuffleWriter[K, V]( protected val mapId: Int = context.partitionId() + protected val clientPushBufferMaxSize: Int = celebornConf.clientPushBufferMaxSize + protected val celebornPartitionPusher = new CelebornPartitionPusher( shuffleId, numMappers, @@ -58,15 +60,15 @@ abstract class CelebornHashBasedColumnarShuffleWriter[K, V]( context, mapId, client, - celebornConf) + clientPushBufferMaxSize) protected val blockManager: BlockManager = SparkEnv.get.blockManager - protected val nativeBufferSize: Int = GlutenConfig.getConf.maxBatchSize + protected val nativeBufferSize: Int = GlutenConfig.getConf.shuffleWriterBufferSize protected val customizedCompressionCodec: String = GlutenShuffleUtils.getCompressionCodec(conf) protected val bufferCompressThreshold: Int = - GlutenConfig.getConf.columnarShuffleBufferCompressThreshold + GlutenConfig.getConf.columnarShuffleCompressionThreshold // Are we in the process of stopping? Because map tasks can call stop() with success = true // and then call stop() with success = false if they get an exception, we want to make sure diff --git a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornPartitionPusher.scala b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornPartitionPusher.scala index 05c581fbd5ba..2f59307230a0 100644 --- a/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornPartitionPusher.scala +++ b/gluten-celeborn/common/src/main/scala/org/apache/spark/shuffle/CelebornPartitionPusher.scala @@ -20,7 +20,6 @@ import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.celeborn.client.ShuffleClient -import org.apache.celeborn.common.CelebornConf import java.io.IOException @@ -31,13 +30,13 @@ class CelebornPartitionPusher( val context: TaskContext, val mapId: Int, val client: ShuffleClient, - val celebornConf: CelebornConf) + val clientPushBufferMaxSize: Int) extends Logging { @throws[IOException] def pushPartitionData(partitionId: Int, buffer: Array[Byte], length: Int): Int = { - logDebug(s"Push record, size ${buffer.length}.") - if (buffer.length > celebornConf.clientPushBufferMaxSize) { + if (length > clientPushBufferMaxSize) { + logDebug(s"Push Data, size $length.") client.pushData( shuffleId, mapId, @@ -49,6 +48,7 @@ class CelebornPartitionPusher( numMappers, numPartitions) } else { + logDebug(s"Merge Data, size $length.") client.mergeData( shuffleId, mapId, diff --git a/gluten-celeborn/package/pom.xml b/gluten-celeborn/package/pom.xml index 6604fc0006cc..3613d308edbe 100644 --- a/gluten-celeborn/package/pom.xml +++ b/gluten-celeborn/package/pom.xml @@ -11,7 +11,7 @@ gluten-celeborn-package jar - Gluten Celeborn Common + Gluten Celeborn Package 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 b707498c0fc1..21be1642ff72 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 @@ -17,7 +17,7 @@ package org.apache.spark.shuffle import io.glutenproject.GlutenConfig -import io.glutenproject.exec.ExecutionCtxs +import io.glutenproject.exec.Runtimes import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers import io.glutenproject.utils.ArrowAbiUtil @@ -62,6 +62,8 @@ private class CelebornColumnarBatchSerializerInstance( extends SerializerInstance with Logging { + private lazy val nmm = NativeMemoryManagers.contextInstance("ShuffleReader") + private lazy val shuffleReaderHandle = { val allocator: BufferAllocator = ArrowBufferAllocators .contextInstance() @@ -83,10 +85,9 @@ private class CelebornColumnarBatchSerializerInstance( .create() .make( cSchema.memoryAddress(), - NativeMemoryManagers.contextInstance("ShuffleReader").getNativeInstanceHandle, + nmm.getNativeInstanceHandle, compressionCodec, - compressionCodecBackend, - GlutenConfig.getConf.columnarShuffleCompressionMode + compressionCodecBackend ) // Close shuffle reader instance as lately as the end of task processing, // since the native reader could hold a reference to memory pool that @@ -104,10 +105,11 @@ private class CelebornColumnarBatchSerializerInstance( new DeserializationStream { private lazy val byteIn: JniByteInputStream = JniByteInputStreams.create(in) private lazy val wrappedOut: GeneralOutIterator = new ColumnarBatchOutIterator( - ExecutionCtxs.contextInstance(), + Runtimes.contextInstance(), ShuffleReaderJniWrapper .create() - .readStream(shuffleReaderHandle, byteIn)) + .readStream(shuffleReaderHandle, byteIn), + nmm) private var cb: ColumnarBatch = _ diff --git a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornHashBasedColumnarShuffleWriter.scala b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornHashBasedColumnarShuffleWriter.scala index 3f6bc3a9a4b2..e6b5efbc97eb 100644 --- a/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornHashBasedColumnarShuffleWriter.scala +++ b/gluten-celeborn/velox/src/main/scala/org/apache/spark/shuffle/VeloxCelebornHashBasedColumnarShuffleWriter.scala @@ -78,7 +78,7 @@ class VeloxCelebornHashBasedColumnarShuffleWriter[K, V]( customizedCompressionCodec, bufferCompressThreshold, GlutenConfig.getConf.columnarShuffleCompressionMode, - celebornConf.clientPushBufferMaxSize, + clientPushBufferMaxSize, celebornPartitionPusher, NativeMemoryManagers .create( diff --git a/gluten-core/pom.xml b/gluten-core/pom.xml index 14fa2bdfabc2..17fd77841425 100644 --- a/gluten-core/pom.xml +++ b/gluten-core/pom.xml @@ -52,7 +52,7 @@ org.apache.spark - spark-hive-thriftserver_${scala.binary.version} + spark-hive_${scala.binary.version} provided diff --git a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/MemoryTargets.java b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/MemoryTargets.java index fa8203718b4a..2484ea8845e5 100644 --- a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/MemoryTargets.java +++ b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/MemoryTargets.java @@ -18,8 +18,7 @@ import io.glutenproject.GlutenConfig; import io.glutenproject.memory.MemoryUsageStatsBuilder; -import io.glutenproject.memory.memtarget.spark.IsolatedMemoryConsumers; -import io.glutenproject.memory.memtarget.spark.RegularMemoryConsumer; +import io.glutenproject.memory.memtarget.spark.TreeMemoryConsumers; import org.apache.spark.memory.TaskMemoryManager; @@ -48,9 +47,12 @@ public static MemoryTarget newConsumer( String name, Spiller spiller, Map virtualChildren) { + final TreeMemoryConsumers.Factory factory; if (GlutenConfig.getConf().memoryIsolation()) { - return IsolatedMemoryConsumers.newConsumer(tmm, name, spiller, virtualChildren); + factory = TreeMemoryConsumers.isolated(); + } else { + factory = TreeMemoryConsumers.shared(); } - return new RegularMemoryConsumer(tmm, name, spiller, virtualChildren); + return factory.newConsumer(tmm, name, spiller, virtualChildren); } } diff --git a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/IsolatedMemoryConsumers.java b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/IsolatedMemoryConsumers.java deleted file mode 100644 index b24764f7822b..000000000000 --- a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/IsolatedMemoryConsumers.java +++ /dev/null @@ -1,67 +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 io.glutenproject.memory.memtarget.spark; - -import io.glutenproject.GlutenConfig; -import io.glutenproject.memory.MemoryUsageStatsBuilder; -import io.glutenproject.memory.memtarget.Spiller; -import io.glutenproject.memory.memtarget.TreeMemoryTarget; - -import org.apache.spark.memory.TaskMemoryManager; - -import java.util.Collections; -import java.util.Map; -import java.util.WeakHashMap; - -/** - * A hub to provide memory target instances whose shared size (in the same task) is limited to X, X - * = executor memory / task slots. - * - *

Using this to prevent OOMs if the delegated memory target could possibly hold large memory - * blocks that are not spillable. - * - *

See GLUTEN-3030 - */ -public class IsolatedMemoryConsumers { - private static final WeakHashMap MAP = new WeakHashMap<>(); - - private IsolatedMemoryConsumers() {} - - private static TreeMemoryTarget getSharedAccount(TaskMemoryManager tmm) { - synchronized (MAP) { - return MAP.computeIfAbsent( - tmm, - m -> { - TreeMemoryTarget tmc = new TreeMemoryConsumer(m); - return tmc.newChild( - "root", - GlutenConfig.getConf().conservativeTaskOffHeapMemorySize(), - Spiller.NO_OP, - Collections.emptyMap()); - }); - } - } - - public static TreeMemoryTarget newConsumer( - TaskMemoryManager tmm, - String name, - Spiller spiller, - Map virtualChildren) { - TreeMemoryTarget account = getSharedAccount(tmm); - return account.newChild(name, TreeMemoryConsumer.CAPACITY_UNLIMITED, spiller, virtualChildren); - } -} diff --git a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/RegularMemoryConsumer.java b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/RegularMemoryConsumer.java index 3131abf46e8c..2dcd6ed0d55c 100644 --- a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/RegularMemoryConsumer.java +++ b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/RegularMemoryConsumer.java @@ -35,7 +35,12 @@ import java.util.Map; import java.util.stream.Collectors; -/** A trivial memory consumer implementation used by Gluten. */ +/** + * A trivial memory consumer implementation used by Gluten. + * + * @deprecated Use {@link TreeMemoryConsumers#shared()} instead. + */ +@Deprecated public class RegularMemoryConsumer extends MemoryConsumer implements MemoryTarget, KnownNameAndStats { diff --git a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumer.java b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumer.java index 410a5fbf138f..2f3d0c700f17 100644 --- a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumer.java +++ b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumer.java @@ -48,7 +48,7 @@ * registered as a Spark memory consumer. * *

Typically used by utility class {@link - * io.glutenproject.memory.memtarget.spark.IsolatedMemoryConsumers}. + * io.glutenproject.memory.memtarget.spark.TreeMemoryConsumers}. */ public class TreeMemoryConsumer extends MemoryConsumer implements TreeMemoryTarget { diff --git a/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumers.java b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumers.java new file mode 100644 index 000000000000..cf851c996666 --- /dev/null +++ b/gluten-core/src/main/java/io/glutenproject/memory/memtarget/spark/TreeMemoryConsumers.java @@ -0,0 +1,92 @@ +/* + * 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 io.glutenproject.memory.memtarget.spark; + +import io.glutenproject.GlutenConfig; +import io.glutenproject.memory.MemoryUsageStatsBuilder; +import io.glutenproject.memory.memtarget.Spiller; +import io.glutenproject.memory.memtarget.TreeMemoryTarget; + +import org.apache.spark.memory.TaskMemoryManager; + +import java.util.Collections; +import java.util.Map; +import java.util.WeakHashMap; +import java.util.concurrent.ConcurrentHashMap; + +public final class TreeMemoryConsumers { + + private static final Map FACTORIES = new ConcurrentHashMap<>(); + + private TreeMemoryConsumers() {} + + private static Factory createOrGetFactory(long perTaskCapacity) { + return FACTORIES.computeIfAbsent(perTaskCapacity, Factory::new); + } + + /** + * A hub to provide memory target instances whose shared size (in the same task) is limited to X, + * X = executor memory / task slots. + * + *

Using this to prevent OOMs if the delegated memory target could possibly hold large memory + * blocks that are not spillable. + * + *

See GLUTEN-3030 + */ + public static Factory isolated() { + return createOrGetFactory(GlutenConfig.getConf().conservativeTaskOffHeapMemorySize()); + } + + /** + * This works as a legacy Spark memory consumer which grants as much as possible of memory + * capacity to each task. + */ + public static Factory shared() { + return createOrGetFactory(TreeMemoryTarget.CAPACITY_UNLIMITED); + } + + public static class Factory { + + private static final WeakHashMap MAP = new WeakHashMap<>(); + private final long perTaskCapacity; + + private Factory(long perTaskCapacity) { + this.perTaskCapacity = perTaskCapacity; + } + + private TreeMemoryTarget getSharedAccount(TaskMemoryManager tmm) { + synchronized (MAP) { + return MAP.computeIfAbsent( + tmm, + m -> { + TreeMemoryTarget tmc = new TreeMemoryConsumer(m); + return tmc.newChild("root", perTaskCapacity, Spiller.NO_OP, Collections.emptyMap()); + }); + } + } + + public TreeMemoryTarget newConsumer( + TaskMemoryManager tmm, + String name, + Spiller spiller, + Map virtualChildren) { + TreeMemoryTarget account = getSharedAccount(tmm); + return account.newChild( + name, TreeMemoryConsumer.CAPACITY_UNLIMITED, spiller, virtualChildren); + } + } +} diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/AggregateFunctionNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/AggregateFunctionNode.java index 7651bfed2815..f7b0659a028b 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/AggregateFunctionNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/AggregateFunctionNode.java @@ -24,16 +24,17 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class AggregateFunctionNode implements Serializable { private final Long functionId; - private final ArrayList expressionNodes = new ArrayList<>(); + private final List expressionNodes = new ArrayList<>(); private final String phase; private final TypeNode outputTypeNode; AggregateFunctionNode( Long functionId, - ArrayList expressionNodes, + List expressionNodes, String phase, TypeNode outputTypeNode) { this.functionId = functionId; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/ExpressionBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/ExpressionBuilder.java index a0f2375d19de..81194fedcef5 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/ExpressionBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/ExpressionBuilder.java @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.*; -import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -239,7 +238,7 @@ public static SelectionNode makeSelection(Integer fieldIdx, Integer childFieldId public static AggregateFunctionNode makeAggregateFunction( Long functionId, - ArrayList expressionNodes, + List expressionNodes, String phase, TypeNode outputTypeNode) { return new AggregateFunctionNode(functionId, expressionNodes, phase, outputTypeNode); @@ -261,7 +260,7 @@ public static SingularOrListNode makeSingularOrListNode( public static WindowFunctionNode makeWindowFunction( Integer functionId, - ArrayList expressionNodes, + List expressionNodes, String columnName, TypeNode outputTypeNode, String upperBound, diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/IfThenNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/IfThenNode.java index fbdca934e26f..40133ccf6af5 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/IfThenNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/IfThenNode.java @@ -20,18 +20,17 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class IfThenNode implements ExpressionNode, Serializable { - private final ArrayList ifNodes = new ArrayList<>(); - private final ArrayList thenNodes = new ArrayList<>(); + private final List ifNodes = new ArrayList<>(); + private final List thenNodes = new ArrayList<>(); private final ExpressionNode elseValue; public IfThenNode( - ArrayList ifNodes, - ArrayList thenNodes, - ExpressionNode elseValue) { + List ifNodes, List thenNodes, ExpressionNode elseValue) { this.ifNodes.addAll(ifNodes); this.thenNodes.addAll(thenNodes); this.elseValue = elseValue; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/SelectionNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/SelectionNode.java index ef8bcd558bd0..7dc5e53cb15c 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/SelectionNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/SelectionNode.java @@ -20,12 +20,13 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class SelectionNode implements ExpressionNode, Serializable { private final Integer fieldIndex; // The nested indices of child field. For case like a.b.c, the index of c is put at last. - private final ArrayList nestedChildIndices = new ArrayList<>(); + private final List nestedChildIndices = new ArrayList<>(); SelectionNode(Integer fieldIndex) { this.fieldIndex = fieldIndex; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/StringMapNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/StringMapNode.java index 62909c171f01..b9ad7120320a 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/StringMapNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/StringMapNode.java @@ -23,7 +23,7 @@ import java.util.Map; public class StringMapNode implements ExpressionNode, Serializable { - private final Map values = new HashMap(); + private final Map values = new HashMap<>(); public StringMapNode(Map values) { this.values.putAll(values); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/expression/WindowFunctionNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/expression/WindowFunctionNode.java index 20716b752676..fa450eeea946 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/expression/WindowFunctionNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/expression/WindowFunctionNode.java @@ -24,10 +24,11 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class WindowFunctionNode implements Serializable { private final Integer functionId; - private final ArrayList expressionNodes = new ArrayList<>(); + private final List expressionNodes = new ArrayList<>(); private final String columnName; private final TypeNode outputTypeNode; @@ -40,7 +41,7 @@ public class WindowFunctionNode implements Serializable { WindowFunctionNode( Integer functionId, - ArrayList expressionNodes, + List expressionNodes, String columnName, TypeNode outputTypeNode, String upperBound, diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanBuilder.java index d9ea9a5f589b..5ea68926088c 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanBuilder.java @@ -24,22 +24,24 @@ import io.glutenproject.substrait.type.TypeNode; import java.util.ArrayList; +import java.util.List; import java.util.Map; public class PlanBuilder { + + public static byte[] EMPTY_PLAN = empty().toProtobuf().toByteArray(); + private PlanBuilder() {} public static PlanNode makePlan( - ArrayList mappingNodes, - ArrayList relNodes, - ArrayList outNames) { + List mappingNodes, List relNodes, List outNames) { return new PlanNode(mappingNodes, relNodes, outNames); } public static PlanNode makePlan( - ArrayList mappingNodes, - ArrayList relNodes, - ArrayList outNames, + List mappingNodes, + List relNodes, + List outNames, TypeNode outputSchema, AdvancedExtensionNode extension) { return new PlanNode(mappingNodes, relNodes, outNames, outputSchema, extension); @@ -50,20 +52,20 @@ public static PlanNode makePlan(AdvancedExtensionNode extension) { } public static PlanNode makePlan( - SubstraitContext subCtx, ArrayList relNodes, ArrayList outNames) { + SubstraitContext subCtx, List relNodes, List outNames) { return makePlan(subCtx, relNodes, outNames, null, null); } public static PlanNode makePlan( SubstraitContext subCtx, - ArrayList relNodes, - ArrayList outNames, + List relNodes, + List outNames, TypeNode outputSchema, AdvancedExtensionNode extension) { if (subCtx == null) { throw new NullPointerException("ColumnarWholestageTransformer cannot doTansform."); } - ArrayList mappingNodes = new ArrayList<>(); + List mappingNodes = new ArrayList<>(); for (Map.Entry entry : subCtx.registeredFunction().entrySet()) { FunctionMappingNode mappingNode = diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanNode.java index c678f7abc696..498dfe7b0f50 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/plan/PlanNode.java @@ -27,28 +27,26 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class PlanNode implements Serializable { - private final ArrayList mappingNodes = new ArrayList<>(); - private final ArrayList relNodes = new ArrayList<>(); - private final ArrayList outNames = new ArrayList<>(); + private final List mappingNodes = new ArrayList<>(); + private final List relNodes = new ArrayList<>(); + private final List outNames = new ArrayList<>(); private TypeNode outputSchema = null; private AdvancedExtensionNode extension = null; - PlanNode( - ArrayList mappingNodes, - ArrayList relNodes, - ArrayList outNames) { + PlanNode(List mappingNodes, List relNodes, List outNames) { this.mappingNodes.addAll(mappingNodes); this.relNodes.addAll(relNodes); this.outNames.addAll(outNames); } PlanNode( - ArrayList mappingNodes, - ArrayList relNodes, - ArrayList outNames, + List mappingNodes, + List relNodes, + List outNames, TypeNode outputSchema, AdvancedExtensionNode extension) { this.mappingNodes.addAll(mappingNodes); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/AggregateRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/AggregateRelNode.java index 75a0931b9074..8db80f9ebc61 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/AggregateRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/AggregateRelNode.java @@ -26,20 +26,21 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class AggregateRelNode implements RelNode, Serializable { private final RelNode input; - private final ArrayList groupings = new ArrayList<>(); - private final ArrayList aggregateFunctionNodes = new ArrayList<>(); + private final List groupings = new ArrayList<>(); + private final List aggregateFunctionNodes = new ArrayList<>(); - private final ArrayList filters = new ArrayList<>(); + private final List filters = new ArrayList<>(); private final AdvancedExtensionNode extensionNode; AggregateRelNode( RelNode input, - ArrayList groupings, - ArrayList aggregateFunctionNodes, - ArrayList filters) { + List groupings, + List aggregateFunctionNodes, + List filters) { this.input = input; this.groupings.addAll(groupings); this.aggregateFunctionNodes.addAll(aggregateFunctionNodes); @@ -49,9 +50,9 @@ public class AggregateRelNode implements RelNode, Serializable { AggregateRelNode( RelNode input, - ArrayList groupings, - ArrayList aggregateFunctionNodes, - ArrayList filters, + List groupings, + List aggregateFunctionNodes, + List filters, AdvancedExtensionNode extensionNode) { this.input = input; this.groupings.addAll(groupings); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExpandRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExpandRelNode.java index 853c1c62b011..49f714b8a7f6 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExpandRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExpandRelNode.java @@ -25,23 +25,22 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class ExpandRelNode implements RelNode, Serializable { private final RelNode input; - private final ArrayList> projections = new ArrayList<>(); + private final List> projections = new ArrayList<>(); private final AdvancedExtensionNode extensionNode; public ExpandRelNode( - RelNode input, - ArrayList> projections, - AdvancedExtensionNode extensionNode) { + RelNode input, List> projections, AdvancedExtensionNode extensionNode) { this.input = input; this.projections.addAll(projections); this.extensionNode = extensionNode; } - public ExpandRelNode(RelNode input, ArrayList> projections) { + public ExpandRelNode(RelNode input, List> projections) { this.input = input; this.projections.addAll(projections); this.extensionNode = null; @@ -59,7 +58,7 @@ public Rel toProtobuf() { expandBuilder.setInput(input.toProtobuf()); } - for (ArrayList projectList : projections) { + for (List projectList : projections) { ExpandRel.ExpandField.Builder expandFieldBuilder = ExpandRel.ExpandField.newBuilder(); ExpandRel.SwitchingField.Builder switchingField = ExpandRel.SwitchingField.newBuilder(); for (ExpressionNode exprNode : projectList) { diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableBuilder.java index f3ff57631b16..c525fa0b5fe3 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableBuilder.java @@ -16,11 +16,19 @@ */ package io.glutenproject.substrait.rel; +import java.util.List; + public class ExtensionTableBuilder { private ExtensionTableBuilder() {} public static ExtensionTableNode makeExtensionTable( - Long minPartsNum, Long maxPartsNum, String database, String tableName, String relativePath) { - return new ExtensionTableNode(minPartsNum, maxPartsNum, database, tableName, relativePath); + Long minPartsNum, + Long maxPartsNum, + String database, + String tableName, + String relativePath, + List preferredLocations) { + return new ExtensionTableNode( + minPartsNum, maxPartsNum, database, tableName, relativePath, preferredLocations); } } diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableNode.java index ad4d40151156..d18d0966c156 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ExtensionTableNode.java @@ -20,24 +20,32 @@ import com.google.protobuf.StringValue; import io.substrait.proto.ReadRel; -import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; -public class ExtensionTableNode implements Serializable { +public class ExtensionTableNode implements SplitInfo { private static final String MERGE_TREE = "MergeTree;"; private Long minPartsNum; private Long maxPartsNum; - private String database = null; - private String tableName = null; - private String relativePath = null; + private String database; + private String tableName; + private String relativePath; private StringBuffer extensionTableStr = new StringBuffer(MERGE_TREE); + private final List preferredLocations = new ArrayList<>(); ExtensionTableNode( - Long minPartsNum, Long maxPartsNum, String database, String tableName, String relativePath) { + Long minPartsNum, + Long maxPartsNum, + String database, + String tableName, + String relativePath, + List preferredLocations) { this.minPartsNum = minPartsNum; this.maxPartsNum = maxPartsNum; this.database = database; this.tableName = tableName; this.relativePath = relativePath; + this.preferredLocations.addAll(preferredLocations); // MergeTree;{database}\n{table}\n{relative_path}\n{min_part}\n{max_part}\n extensionTableStr .append(database) @@ -52,6 +60,11 @@ public class ExtensionTableNode implements Serializable { .append("\n"); } + @Override + public List preferredLocations() { + return this.preferredLocations; + } + public ReadRel.ExtensionTable toProtobuf() { ReadRel.ExtensionTable.Builder extensionTableBuilder = ReadRel.ExtensionTable.newBuilder(); StringValue extensionTable = diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/GenerateRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/GenerateRelNode.java index 828931c4a38d..4a53b2bd08f7 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/GenerateRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/GenerateRelNode.java @@ -24,22 +24,22 @@ import io.substrait.proto.RelCommon; import java.io.Serializable; -import java.util.ArrayList; +import java.util.List; public class GenerateRelNode implements RelNode, Serializable { private final RelNode input; private final ExpressionNode generator; - private final ArrayList childOutput; + private final List childOutput; private final AdvancedExtensionNode extensionNode; - GenerateRelNode(RelNode input, ExpressionNode generator, ArrayList childOutput) { + GenerateRelNode(RelNode input, ExpressionNode generator, List childOutput) { this(input, generator, childOutput, null); } GenerateRelNode( RelNode input, ExpressionNode generator, - ArrayList childOutput, + List childOutput, AdvancedExtensionNode extensionNode) { this.input = input; this.generator = generator; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesBuilder.java index be5d56de4b28..c86c90cc667a 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesBuilder.java @@ -28,8 +28,10 @@ public static LocalFilesNode makeLocalFiles( List starts, List lengths, List> partitionColumns, - LocalFilesNode.ReadFileFormat fileFormat) { - return new LocalFilesNode(index, paths, starts, lengths, partitionColumns, fileFormat); + LocalFilesNode.ReadFileFormat fileFormat, + List preferredLocations) { + return new LocalFilesNode( + index, paths, starts, lengths, partitionColumns, fileFormat, preferredLocations); } public static LocalFilesNode makeLocalFiles(String iterPath) { diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesNode.java index b0a1c5c8d792..f781ed2b04e3 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/LocalFilesNode.java @@ -25,17 +25,17 @@ import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.Map; -public class LocalFilesNode implements Serializable { +public class LocalFilesNode implements SplitInfo { private final Integer index; private final List paths = new ArrayList<>(); private final List starts = new ArrayList<>(); private final List lengths = new ArrayList<>(); private final List> partitionColumns = new ArrayList<>(); + private final List preferredLocations = new ArrayList<>(); // The format of file to read. public enum ReadFileFormat { @@ -60,13 +60,15 @@ public enum ReadFileFormat { List starts, List lengths, List> partitionColumns, - ReadFileFormat fileFormat) { + ReadFileFormat fileFormat, + List preferredLocations) { this.index = index; this.paths.addAll(paths); this.starts.addAll(starts); this.lengths.addAll(lengths); this.fileFormat = fileFormat; this.partitionColumns.addAll(partitionColumns); + this.preferredLocations.addAll(preferredLocations); } LocalFilesNode(String iterPath) { @@ -98,6 +100,11 @@ public void setFileReadProperties(Map fileReadProperties) { this.fileReadProperties = fileReadProperties; } + @Override + public List preferredLocations() { + return this.preferredLocations; + } + public ReadRel.LocalFiles toProtobuf() { ReadRel.LocalFiles.Builder localFilesBuilder = ReadRel.LocalFiles.newBuilder(); // The input is iterator, and the path is in the format of: Iterator:index. diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ProjectRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ProjectRelNode.java index 595b823d38f2..ee88390c74ed 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ProjectRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ProjectRelNode.java @@ -25,14 +25,15 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class ProjectRelNode implements RelNode, Serializable { private final RelNode input; - private final ArrayList expressionNodes = new ArrayList<>(); + private final List expressionNodes = new ArrayList<>(); private final AdvancedExtensionNode extensionNode; private final int emitStartIndex; - ProjectRelNode(RelNode input, ArrayList expressionNodes, int emitStartIndex) { + ProjectRelNode(RelNode input, List expressionNodes, int emitStartIndex) { this.input = input; this.expressionNodes.addAll(expressionNodes); this.extensionNode = null; @@ -41,7 +42,7 @@ public class ProjectRelNode implements RelNode, Serializable { ProjectRelNode( RelNode input, - ArrayList expressionNodes, + List expressionNodes, AdvancedExtensionNode extensionNode, int emitStartIndex) { this.input = input; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ReadRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ReadRelNode.java index 9fb47b9e14cf..a28f72427ab0 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/ReadRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/ReadRelNode.java @@ -31,12 +31,13 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; import java.util.Map; public class ReadRelNode implements RelNode, Serializable { - private final ArrayList types = new ArrayList<>(); - private final ArrayList names = new ArrayList<>(); - private final ArrayList columnTypeNodes = new ArrayList<>(); + private final List types = new ArrayList<>(); + private final List names = new ArrayList<>(); + private final List columnTypeNodes = new ArrayList<>(); private final SubstraitContext context; private final ExpressionNode filterNode; private final Long iteratorIndex; @@ -44,8 +45,8 @@ public class ReadRelNode implements RelNode, Serializable { private Map properties; ReadRelNode( - ArrayList types, - ArrayList names, + List types, + List names, SubstraitContext context, ExpressionNode filterNode, Long iteratorIndex) { @@ -57,12 +58,12 @@ public class ReadRelNode implements RelNode, Serializable { } ReadRelNode( - ArrayList types, - ArrayList names, + List types, + List names, SubstraitContext context, ExpressionNode filterNode, Long iteratorIndex, - ArrayList columnTypeNodes) { + List columnTypeNodes) { this.types.addAll(types); this.names.addAll(names); this.context = context; @@ -131,17 +132,17 @@ public Rel toProtobuf() { filesNode.setFileReadProperties(properties); } readBuilder.setLocalFiles(filesNode.toProtobuf()); - } else if (context.getLocalFilesNodes() != null && !context.getLocalFilesNodes().isEmpty()) { - Serializable currentLocalFileNode = context.getCurrentLocalFileNode(); - if (currentLocalFileNode instanceof LocalFilesNode) { - LocalFilesNode filesNode = (LocalFilesNode) currentLocalFileNode; + } else if (context.getSplitInfos() != null && !context.getSplitInfos().isEmpty()) { + SplitInfo currentSplitInfo = context.getCurrentSplitInfo(); + if (currentSplitInfo instanceof LocalFilesNode) { + LocalFilesNode filesNode = (LocalFilesNode) currentSplitInfo; if (dataSchema != null) { filesNode.setFileSchema(dataSchema); filesNode.setFileReadProperties(properties); } - readBuilder.setLocalFiles(((LocalFilesNode) currentLocalFileNode).toProtobuf()); - } else if (currentLocalFileNode instanceof ExtensionTableNode) { - readBuilder.setExtensionTable(((ExtensionTableNode) currentLocalFileNode).toProtobuf()); + readBuilder.setLocalFiles(((LocalFilesNode) currentSplitInfo).toProtobuf()); + } else if (currentSplitInfo instanceof ExtensionTableNode) { + readBuilder.setExtensionTable(((ExtensionTableNode) currentSplitInfo).toProtobuf()); } } Rel.Builder builder = Rel.newBuilder(); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/RelBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/RelBuilder.java index 681c4ecf42b4..8dfb2f4a20af 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/RelBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/RelBuilder.java @@ -29,7 +29,7 @@ import io.substrait.proto.SortField; import org.apache.spark.sql.catalyst.expressions.Attribute; -import java.util.ArrayList; +import java.util.List; /** Contains helper functions for constructing substrait relations. */ public class RelBuilder { @@ -53,7 +53,7 @@ public static RelNode makeFilterRel( public static RelNode makeProjectRel( RelNode input, - ArrayList expressionNodes, + List expressionNodes, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); @@ -62,7 +62,7 @@ public static RelNode makeProjectRel( public static RelNode makeProjectRel( RelNode input, - ArrayList expressionNodes, + List expressionNodes, SubstraitContext context, Long operatorId, int emitStartIndex) { @@ -72,7 +72,7 @@ public static RelNode makeProjectRel( public static RelNode makeProjectRel( RelNode input, - ArrayList expressionNodes, + List expressionNodes, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId, @@ -83,9 +83,9 @@ public static RelNode makeProjectRel( public static RelNode makeAggregateRel( RelNode input, - ArrayList groupings, - ArrayList aggregateFunctionNodes, - ArrayList filters, + List groupings, + List aggregateFunctionNodes, + List filters, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); @@ -94,9 +94,9 @@ public static RelNode makeAggregateRel( public static RelNode makeAggregateRel( RelNode input, - ArrayList groupings, - ArrayList aggregateFunctionNodes, - ArrayList filters, + List groupings, + List aggregateFunctionNodes, + List filters, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId) { @@ -105,8 +105,8 @@ public static RelNode makeAggregateRel( } public static RelNode makeReadRel( - ArrayList types, - ArrayList names, + List types, + List names, ExpressionNode filter, SubstraitContext context, Long operatorId) { @@ -115,9 +115,9 @@ public static RelNode makeReadRel( } public static RelNode makeReadRel( - ArrayList types, - ArrayList names, - ArrayList columnTypeNodes, + List types, + List names, + List columnTypeNodes, ExpressionNode filter, SubstraitContext context, Long operatorId) { @@ -126,8 +126,8 @@ public static RelNode makeReadRel( } public static RelNode makeReadRel( - ArrayList types, - ArrayList names, + List types, + List names, ExpressionNode filter, Long iteratorIndex, SubstraitContext context, @@ -137,15 +137,15 @@ public static RelNode makeReadRel( } public static RelNode makeReadRel( - ArrayList attributes, SubstraitContext context, Long operatorId) { + List attributes, SubstraitContext context, Long operatorId) { if (operatorId >= 0) { // If the operator id is negative, will not register the rel to operator. // Currently, only for the special handling in join. context.registerRelToOperator(operatorId); } - ArrayList typeList = ConverterUtils.collectAttributeTypeNodes(attributes); - ArrayList nameList = ConverterUtils.collectAttributeNamesWithExprId(attributes); + List typeList = ConverterUtils.collectAttributeTypeNodes(attributes); + List nameList = ConverterUtils.collectAttributeNamesWithExprId(attributes); // The iterator index will be added in the path of LocalFiles. Long iteratorIndex = context.nextIteratorIndex(); @@ -184,7 +184,7 @@ public static RelNode makeJoinRel( public static RelNode makeExpandRel( RelNode input, - ArrayList> projections, + List> projections, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId) { @@ -194,7 +194,7 @@ public static RelNode makeExpandRel( public static RelNode makeExpandRel( RelNode input, - ArrayList> projections, + List> projections, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); @@ -203,7 +203,7 @@ public static RelNode makeExpandRel( public static RelNode makeSortRel( RelNode input, - ArrayList sorts, + List sorts, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId) { @@ -212,7 +212,7 @@ public static RelNode makeSortRel( } public static RelNode makeSortRel( - RelNode input, ArrayList sorts, SubstraitContext context, Long operatorId) { + RelNode input, List sorts, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); return new SortRelNode(input, sorts); } @@ -236,9 +236,9 @@ public static RelNode makeFetchRel( public static RelNode makeWindowRel( RelNode input, - ArrayList windowFunctionNodes, - ArrayList partitionExpressions, - ArrayList sorts, + List windowFunctionNodes, + List partitionExpressions, + List sorts, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId) { @@ -249,9 +249,9 @@ public static RelNode makeWindowRel( public static RelNode makeWindowRel( RelNode input, - ArrayList windowFunctionNodes, - ArrayList partitionExpressions, - ArrayList sorts, + List windowFunctionNodes, + List partitionExpressions, + List sorts, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); @@ -263,7 +263,7 @@ public static RelNode makeWindowRel( public static RelNode makeGenerateRel( RelNode input, ExpressionNode generator, - ArrayList childOutput, + List childOutput, SubstraitContext context, Long operatorId) { context.registerRelToOperator(operatorId); @@ -273,7 +273,7 @@ public static RelNode makeGenerateRel( public static RelNode makeGenerateRel( RelNode input, ExpressionNode generator, - ArrayList childOutput, + List childOutput, AdvancedExtensionNode extensionNode, SubstraitContext context, Long operatorId) { diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/SortRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/SortRelNode.java index 2fd129d5045e..fb2b8b128e04 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/SortRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/SortRelNode.java @@ -25,20 +25,20 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class SortRelNode implements RelNode, Serializable { private final RelNode input; - private final ArrayList sorts = new ArrayList<>(); + private final List sorts = new ArrayList<>(); private final AdvancedExtensionNode extensionNode; - public SortRelNode( - RelNode input, ArrayList sorts, AdvancedExtensionNode extensionNode) { + public SortRelNode(RelNode input, List sorts, AdvancedExtensionNode extensionNode) { this.input = input; this.sorts.addAll(sorts); this.extensionNode = extensionNode; } - public SortRelNode(RelNode input, ArrayList sorts) { + public SortRelNode(RelNode input, List sorts) { this.input = input; this.sorts.addAll(sorts); this.extensionNode = null; diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/SplitInfo.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/SplitInfo.java new file mode 100644 index 000000000000..42125a253979 --- /dev/null +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/SplitInfo.java @@ -0,0 +1,34 @@ +/* + * 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 io.glutenproject.substrait.rel; + +import com.google.protobuf.MessageOrBuilder; + +import java.io.Serializable; +import java.util.List; + +/** + * A serializable representation of a read split for native engine, including the file path and + * other information of the scan table. It is returned by {@link + * io.glutenproject.execution.BasicScanExecTransformer#getSplitInfos()}. + */ +public interface SplitInfo extends Serializable { + /** The preferred locations where the table files returned by this read split can run faster. */ + List preferredLocations(); + + MessageOrBuilder toProtobuf(); +} diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/rel/WindowRelNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/rel/WindowRelNode.java index 475f9f6f3cd0..f120c37c73f4 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/rel/WindowRelNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/rel/WindowRelNode.java @@ -27,19 +27,20 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class WindowRelNode implements RelNode, Serializable { private final RelNode input; - private final ArrayList windowFunctionNodes = new ArrayList<>(); - private final ArrayList partitionExpressions = new ArrayList<>(); - private final ArrayList sorts = new ArrayList<>(); + private final List windowFunctionNodes = new ArrayList<>(); + private final List partitionExpressions = new ArrayList<>(); + private final List sorts = new ArrayList<>(); private final AdvancedExtensionNode extensionNode; public WindowRelNode( RelNode input, - ArrayList windowFunctionNodes, - ArrayList partitionExpressions, - ArrayList sorts) { + List windowFunctionNodes, + List partitionExpressions, + List sorts) { this.input = input; this.windowFunctionNodes.addAll(windowFunctionNodes); this.partitionExpressions.addAll(partitionExpressions); @@ -49,9 +50,9 @@ public WindowRelNode( public WindowRelNode( RelNode input, - ArrayList windowFunctionNodes, - ArrayList partitionExpressions, - ArrayList sorts, + List windowFunctionNodes, + List partitionExpressions, + List sorts, AdvancedExtensionNode extensionNode) { this.input = input; this.windowFunctionNodes.addAll(windowFunctionNodes); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/type/MapNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/type/MapNode.java index 35ede7ed0950..76c4c76734cc 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/type/MapNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/type/MapNode.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class MapNode implements TypeNode, Serializable { private final Boolean nullable; @@ -34,7 +35,7 @@ public MapNode(Boolean nullable, TypeNode keyType, TypeNode valType) { // It's used in ExplodeTransformer to determine output datatype from children. public TypeNode getNestedType() { - ArrayList types = new ArrayList<>(); + List types = new ArrayList<>(); types.add(keyType); types.add(valType); return TypeBuilder.makeStruct(false, types); diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/type/StructNode.java b/gluten-core/src/main/java/io/glutenproject/substrait/type/StructNode.java index 8b81b330518f..ff7c0922d822 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/type/StructNode.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/type/StructNode.java @@ -20,24 +20,25 @@ import java.io.Serializable; import java.util.ArrayList; +import java.util.List; public class StructNode implements TypeNode, Serializable { private final Boolean nullable; - private final ArrayList types = new ArrayList<>(); - private final ArrayList names = new ArrayList<>(); + private final List types = new ArrayList<>(); + private final List names = new ArrayList<>(); - public StructNode(Boolean nullable, ArrayList types, ArrayList names) { + public StructNode(Boolean nullable, List types, List names) { this.nullable = nullable; this.types.addAll(types); this.names.addAll(names); } - public StructNode(Boolean nullable, ArrayList types) { + public StructNode(Boolean nullable, List types) { this.nullable = nullable; this.types.addAll(types); } - public ArrayList getFieldTypes() { + public List getFieldTypes() { return types; } diff --git a/gluten-core/src/main/java/io/glutenproject/substrait/type/TypeBuilder.java b/gluten-core/src/main/java/io/glutenproject/substrait/type/TypeBuilder.java index bdaed9323825..a3efd67653c3 100644 --- a/gluten-core/src/main/java/io/glutenproject/substrait/type/TypeBuilder.java +++ b/gluten-core/src/main/java/io/glutenproject/substrait/type/TypeBuilder.java @@ -16,7 +16,7 @@ */ package io.glutenproject.substrait.type; -import java.util.ArrayList; +import java.util.List; public class TypeBuilder { private TypeBuilder() {} @@ -77,12 +77,11 @@ public static TypeNode makeTimestamp(Boolean nullable) { return new TimestampTypeNode(nullable); } - public static TypeNode makeStruct( - Boolean nullable, ArrayList types, ArrayList names) { + public static TypeNode makeStruct(Boolean nullable, List types, List names) { return new StructNode(nullable, types, names); } - public static TypeNode makeStruct(Boolean nullable, ArrayList types) { + public static TypeNode makeStruct(Boolean nullable, List types) { return new StructNode(nullable, types); } diff --git a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIterator.java b/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIterator.java index 48e2e9a6de70..853330168b86 100644 --- a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIterator.java +++ b/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIterator.java @@ -22,27 +22,19 @@ public abstract class GeneralInIterator implements AutoCloseable { protected final Iterator delegated; - private transient ColumnarBatch nextBatch = null; public GeneralInIterator(Iterator delegated) { this.delegated = delegated; } public boolean hasNext() { - while (delegated.hasNext()) { - nextBatch = delegated.next(); - if (nextBatch.numRows() > 0) { - // any problem using delegated.hasNext() instead? - return true; - } - } - return false; + return delegated.hasNext(); } @Override public void close() throws Exception {} public ColumnarBatch nextColumnarBatch() { - return nextBatch; + return delegated.next(); } } diff --git a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralOutIterator.java b/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralOutIterator.java index 216f00c21d32..28b857ef2a42 100644 --- a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralOutIterator.java +++ b/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralOutIterator.java @@ -16,24 +16,37 @@ */ package io.glutenproject.vectorized; +import io.glutenproject.exception.GlutenException; import io.glutenproject.metrics.IMetrics; import org.apache.spark.sql.vectorized.ColumnarBatch; import java.io.Serializable; +import java.util.Iterator; import java.util.concurrent.atomic.AtomicBoolean; -public abstract class GeneralOutIterator implements AutoCloseable, Serializable { +public abstract class GeneralOutIterator + implements AutoCloseable, Serializable, Iterator { protected final AtomicBoolean closed = new AtomicBoolean(false); public GeneralOutIterator() {} - public final boolean hasNext() throws Exception { - return hasNextInternal(); + @Override + public final boolean hasNext() { + try { + return hasNextInternal(); + } catch (Exception e) { + throw new GlutenException(e); + } } - public final ColumnarBatch next() throws Exception { - return nextInternal(); + @Override + public final ColumnarBatch next() { + try { + return nextInternal(); + } catch (Exception e) { + throw new GlutenException(e); + } } public final IMetrics getMetrics() throws Exception { diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala index 461faccdb13f..fcd1bbfe8453 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/BackendSettingsApi.scala @@ -87,6 +87,8 @@ trait BackendSettingsApi { /** Get the config prefix for each backend */ def getBackendConfigPrefix: String + def allowDecimalArithmetic: Boolean = true + def rescaleDecimalIntegralExpression(): Boolean = false def shuffleSupportedCodec(): Set[String] @@ -96,8 +98,6 @@ trait BackendSettingsApi { /** Apply necessary conversions before passing to native side */ def resolveNativeConf(nativeConf: java.util.Map[String, String]): Unit = {} - def supportBucketScan(): Boolean = false - def insertPostProjectForGenerate(): Boolean = false def skipNativeCtas(ctas: CreateDataSourceTableAsSelectCommand): Boolean = false diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/IteratorApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/IteratorApi.scala index 13ea170c6378..30b4b28835e5 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/IteratorApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/IteratorApi.scala @@ -21,6 +21,7 @@ import io.glutenproject.execution.{BaseGlutenPartition, BroadCastHashJoinContext import io.glutenproject.metrics.IMetrics import io.glutenproject.substrait.plan.PlanNode import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat +import io.glutenproject.substrait.rel.SplitInfo import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -38,19 +39,10 @@ trait IteratorApi { * * @return */ - def genFilePartition( - index: Int, - partitions: Seq[InputPartition], - partitionSchema: Seq[StructType], - fileFormats: Seq[ReadFileFormat], - wsCxt: WholeStageTransformContext): BaseGlutenPartition - - /** - * Generate closeable ColumnBatch iterator. - * - * @return - */ - def genCloseableColumnBatchIterator(iter: Iterator[ColumnarBatch]): Iterator[ColumnarBatch] + def genSplitInfo( + partition: InputPartition, + partitionSchemas: StructType, + fileFormat: ReadFileFormat): SplitInfo /** * Generate Iterator[ColumnarBatch] for first stage. ("first" means it does not depend on other @@ -89,16 +81,13 @@ trait IteratorApi { def genNativeFileScanRDD( sparkContext: SparkContext, wsCxt: WholeStageTransformContext, - fileFormat: ReadFileFormat, - inputPartitions: Seq[InputPartition], + splitInfos: Seq[SplitInfo], numOutputRows: SQLMetric, numOutputBatches: SQLMetric, scanTime: SQLMetric): RDD[ColumnarBatch] /** Compute for BroadcastBuildSideRDD */ def genBroadcastBuildSideIterator( - split: Partition, - context: TaskContext, broadcasted: broadcast.Broadcast[BuildSideRelation], broadCastContext: BroadCastHashJoinContext): Iterator[ColumnarBatch] } diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/MetricsApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/MetricsApi.scala index 9c24c1284d84..2425c9fc6f51 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/MetricsApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/MetricsApi.scala @@ -23,6 +23,9 @@ import org.apache.spark.SparkContext import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import java.lang.{Long => JLong} +import java.util.{List => JList, Map => JMap} + trait MetricsApi extends Serializable { def genWholeStageTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] = @@ -30,9 +33,9 @@ trait MetricsApi extends Serializable { def metricsUpdatingFunction( child: SparkPlan, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): IMetrics => Unit + relMap: JMap[JLong, JList[JLong]], + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit def genBatchScanTransformerMetrics(sparkContext: SparkContext): Map[String, SQLMetric] diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala index 3f0790ffca09..f910e16d9a0b 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/SparkPlanExecApi.scala @@ -41,7 +41,10 @@ import org.apache.spark.sql.hive.HiveTableScanExecTransformer import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch -import java.util +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} + +import scala.collection.JavaConverters._ trait SparkPlanExecApi { @@ -134,7 +137,7 @@ trait SparkPlanExecApi { /** Transform GetArrayItem to Substrait. */ def genGetArrayItemExpressionNode( substraitExprName: String, - functionMap: java.util.HashMap[String, java.lang.Long], + functionMap: JMap[String, JLong], leftNode: ExpressionNode, rightNode: ExpressionNode, original: GetArrayItem): ExpressionNode @@ -251,6 +254,18 @@ trait SparkPlanExecApi { GenericExpressionTransformer(substraitExprName, Seq(child), original) } + def genStringTranslateTransformer( + substraitExprName: String, + srcExpr: ExpressionTransformer, + matchingExpr: ExpressionTransformer, + replaceExpr: ExpressionTransformer, + original: StringTranslate): ExpressionTransformer = { + GenericExpressionTransformer( + substraitExprName, + Seq(srcExpr, matchingExpr, replaceExpr), + original) + } + def genStringLocateTransformer( substraitExprName: String, first: ExpressionTransformer, @@ -342,9 +357,9 @@ trait SparkPlanExecApi { /** default function to generate window function node */ def genWindowFunctionsNode( windowExpression: Seq[NamedExpression], - windowExpressionNodes: util.ArrayList[WindowFunctionNode], + windowExpressionNodes: JList[WindowFunctionNode], originalInputAttributes: Seq[Attribute], - args: util.HashMap[String, java.lang.Long]): Unit = { + args: JMap[String, JLong]): Unit = { windowExpression.map { windowExpr => @@ -357,7 +372,7 @@ trait SparkPlanExecApi { val frame = aggWindowFunc.frame.asInstanceOf[SpecifiedWindowFrame] val windowFunctionNode = ExpressionBuilder.makeWindowFunction( WindowFunctionsBuilder.create(args, aggWindowFunc).toInt, - new util.ArrayList[ExpressionNode](), + new JArrayList[ExpressionNode](), columnName, ConverterUtils.getTypeNode(aggWindowFunc.dataType, aggWindowFunc.nullable), WindowExecTransformer.getFrameBound(frame.upper), @@ -373,13 +388,12 @@ trait SparkPlanExecApi { throw new UnsupportedOperationException(s"Not currently supported: $aggregateFunc.") } - val childrenNodeList = new util.ArrayList[ExpressionNode]() - aggregateFunc.children.foreach( - expr => - childrenNodeList.add( - ExpressionConverter - .replaceWithExpressionTransformer(expr, originalInputAttributes) - .doTransform(args))) + val childrenNodeList = aggregateFunc.children + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args)) + .asJava val windowFunctionNode = ExpressionBuilder.makeWindowFunction( AggregateFunctionsBuilder.create(args, aggExpression.aggregateFunction).toInt, @@ -394,7 +408,7 @@ trait SparkPlanExecApi { case wf @ (Lead(_, _, _, _) | Lag(_, _, _, _)) => val offset_wf = wf.asInstanceOf[FrameLessOffsetWindowFunction] val frame = offset_wf.frame.asInstanceOf[SpecifiedWindowFrame] - val childrenNodeList = new util.ArrayList[ExpressionNode]() + val childrenNodeList = new JArrayList[ExpressionNode]() childrenNodeList.add( ExpressionConverter .replaceWithExpressionTransformer( @@ -425,12 +439,12 @@ trait SparkPlanExecApi { windowExpressionNodes.add(windowFunctionNode) case wf @ NthValue(input, offset: Literal, _) => val frame = wExpression.windowSpec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] - val childrenNodeList = new util.ArrayList[ExpressionNode]() + val childrenNodeList = new JArrayList[ExpressionNode]() childrenNodeList.add( ExpressionConverter .replaceWithExpressionTransformer(input, attributeSeq = originalInputAttributes) .doTransform(args)) - childrenNodeList.add(new LiteralTransformer(offset).doTransform(args)) + childrenNodeList.add(LiteralTransformer(offset).doTransform(args)) val windowFunctionNode = ExpressionBuilder.makeWindowFunction( WindowFunctionsBuilder.create(args, wf).toInt, childrenNodeList, diff --git a/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala b/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala index cfdd8e376196..30fd9bfde2a1 100644 --- a/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala +++ b/gluten-core/src/main/scala/io/glutenproject/backendsapi/TransformerApi.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.types.DecimalType import org.apache.spark.util.collection.BitSet import java.util @@ -44,6 +45,7 @@ trait TransformerApi { relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean): Seq[InputPartition] @@ -73,4 +75,14 @@ trait TransformerApi { left: ExpressionNode, right: ExpressionNode, escapeChar: ExpressionNode): Iterable[ExpressionNode] + + def createCheckOverflowExprNode( + args: java.lang.Object, + substraitExprName: String, + childNode: ExpressionNode, + dataType: DecimalType, + nullable: Boolean, + nullOnOverflow: Boolean): ExpressionNode + + def getNativePlanString(substraitPlan: Array[Byte], details: Boolean): String } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala index b2a13b89a129..74ded1bdc224 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/BasicPhysicalOperatorTransformer.scala @@ -16,15 +16,14 @@ */ package io.glutenproject.execution -import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer} import io.glutenproject.extension.{GlutenPlan, ValidationResult} import io.glutenproject.extension.columnar.TransformHints import io.glutenproject.metrics.MetricsUpdater -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.sql.shims.SparkShimLoader +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.substrait.expression.ExpressionNode import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} @@ -39,15 +38,11 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.Any -import java.util - import scala.collection.JavaConverters._ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkPlan) - extends UnaryExecNode - with TransformSupport + extends UnaryTransformSupport with PredicateHelper - with AliasAwareOutputPartitioning with Logging { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @@ -65,34 +60,11 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP // The columns that will filtered out by `IsNotNull` could be considered as not nullable. private val notNullAttributes = notNullPreds.flatMap(_.references).distinct.map(_.exprId) - override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator - override def isNullIntolerant(expr: Expression): Boolean = expr match { case e: NullIntolerant => e.children.forall(isNullIntolerant) case _ => false } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - c.getBuildPlans - case _ => - Seq() - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genFilterTransformerMetricsUpdater(metrics) @@ -119,18 +91,15 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP RelBuilder.makeFilterRel(input, condExprNode, context, operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeFilterRel(input, condExprNode, extensionNode, context, operatorId) } } - override protected def outputExpressions: Seq[NamedExpression] = output - override def output: Seq[Attribute] = { child.output.map { a => @@ -187,13 +156,12 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute](child.output.asJava) getRelNode( context, cond, child.output, operatorId, - RelBuilder.makeReadRel(attrList, context, operatorId), + RelBuilder.makeReadRel(child.output.asJava, context, operatorId), validation = false) } assert(currRel != null, "Filter rel should be valid.") @@ -208,18 +176,11 @@ abstract class FilterExecTransformerBase(val cond: Expression, val input: SparkP } TransformContext(inputAttributes, output, currRel) } - - override protected def doExecute() - : org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = { - throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") - } } case class ProjectExecTransformer private (projectList: Seq[NamedExpression], child: SparkPlan) - extends UnaryExecNode - with TransformSupport + extends UnaryTransformSupport with PredicateHelper - with AliasAwareOutputPartitioning with Logging { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @@ -228,8 +189,6 @@ case class ProjectExecTransformer private (projectList: Seq[NamedExpression], ch val sparkConf: SparkConf = sparkContext.getConf - override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator - override protected def doValidateInternal(): ValidationResult = { val substraitContext = new SubstraitContext // Firstly, need to check if the Substrait plan for this operator can be successfully generated. @@ -245,27 +204,6 @@ case class ProjectExecTransformer private (projectList: Seq[NamedExpression], ch case _ => false } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - c.getBuildPlans - case _ => - Seq() - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genProjectTransformerMetricsUpdater(metrics) @@ -298,11 +236,7 @@ case class ProjectExecTransformer private (projectList: Seq[NamedExpression], ch } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) ( getRelNode(context, projectList, child.output, operatorId, readRel, validation = false), child.output) @@ -325,23 +259,18 @@ case class ProjectExecTransformer private (projectList: Seq[NamedExpression], ch validation: Boolean): RelNode = { val args = context.registeredFunction val columnarProjExprs: Seq[ExpressionTransformer] = projectList.map( - expr => { + expr => ExpressionConverter - .replaceWithExpressionTransformer(expr, attributeSeq = originalInputAttributes) - }) - val projExprNodeList = new java.util.ArrayList[ExpressionNode]() - for (expr <- columnarProjExprs) { - projExprNodeList.add(expr.doTransform(args)) - } + .replaceWithExpressionTransformer(expr, attributeSeq = originalInputAttributes)) + val projExprNodeList = columnarProjExprs.map(_.doTransform(args)).asJava val emitStartIndex = originalInputAttributes.size if (!validation) { RelBuilder.makeProjectRel(input, projExprNodeList, context, operatorId, emitStartIndex) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeProjectRel( @@ -358,57 +287,51 @@ case class ProjectExecTransformer private (projectList: Seq[NamedExpression], ch throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } - override protected def outputExpressions: Seq[NamedExpression] = projectList - - override protected def doExecute() - : org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = { - throw new UnsupportedOperationException(s"This operator doesn't support doExecute().") - } - override protected def withNewChildInternal(newChild: SparkPlan): ProjectExecTransformer = copy(child = newChild) } object ProjectExecTransformer { private def processProjectExecTransformer( - project: ProjectExecTransformer): ProjectExecTransformer = { - // Special treatment for Project containing all bloom filters: - // If more than one bloom filter, Spark will merge them into a named_struct - // and then broadcast. The named_struct looks like {'bloomFilter',BF1,'bloomFilter',BF2}, - // with two bloom filters sharing same name. This will cause problem for some backends, - // e.g. ClickHouse, which cannot tolerate duplicate type names in struct type - // So we need to rename 'bloomFilter' to make them unique. - if (project.projectList.size == 1) { - val newProjectListHead = project.projectList.head match { - case alias @ Alias(cns @ CreateNamedStruct(children: Seq[Expression]), "mergedValue") => - if ( - !cns.nameExprs.forall( - e => - e.isInstanceOf[Literal] && "bloomFilter".equals(e.asInstanceOf[Literal].toString())) - ) { - null - } else { - val newChildren = children.zipWithIndex.map { - case _ @(_: Literal, index) => - val newLiteral = Literal("bloomFilter" + index / 2) - newLiteral - case other @ (_, _) => other._1 + projectList: Seq[NamedExpression]): Seq[NamedExpression] = { + + // When there is a MergeScalarSubqueries which will create the named_struct with the same name, + // looks like {'bloomFilter', BF1, 'bloomFilter', BF2} + // or {'count(1)', count(1)#111L, 'avg(a)', avg(a)#222L, 'count(1)', count(1)#333L}, + // it will cause problem for some backends, e.g. ClickHouse, + // which cannot tolerate duplicate type names in struct type, + // so we need to rename 'nameExpr' in the named_struct to make them unique + // after executing the MergeScalarSubqueries. + var needToReplace = false + val newProjectList = projectList.map { + case alias @ Alias(cns @ CreateNamedStruct(children: Seq[Expression]), "mergedValue") => + // check whether there are some duplicate names + if (cns.nameExprs.distinct.size == cns.nameExprs.size) { + alias + } else { + val newChildren = children + .grouped(2) + .flatMap { + case Seq(name: Literal, value: NamedExpression) => + val newLiteral = Literal(name.toString() + "#" + value.exprId.id) + Seq(newLiteral, value) + case Seq(name, value) => Seq(name, value) } - Alias.apply(CreateNamedStruct(newChildren), "mergedValue")(alias.exprId) - } - case _ => null - } - if (newProjectListHead == null) { - project - } else { - ProjectExecTransformer(Seq(newProjectListHead), project.child) - } + .toSeq + needToReplace = true + Alias.apply(CreateNamedStruct(newChildren), "mergedValue")(alias.exprId) + } + case other: NamedExpression => other + } + + if (!needToReplace) { + projectList } else { - project + newProjectList } } def apply(projectList: Seq[NamedExpression], child: SparkPlan): ProjectExecTransformer = - processProjectExecTransformer(new ProjectExecTransformer(projectList, child)) + new ProjectExecTransformer(processProjectExecTransformer(projectList), child) } // An alternatives for UnionExec. @@ -564,7 +487,11 @@ object FilterHandler { getLeftFilters(scan.dataFilters, flattenCondition(plan.condition)) val newPartitionFilters = ExpressionConverter.transformDynamicPruningExpr(scan.partitionFilters, reuseSubquery) - new BatchScanExecTransformer(batchScan.output, scan, leftFilters ++ newPartitionFilters) + new BatchScanExecTransformer( + batchScan.output, + scan, + leftFilters ++ newPartitionFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(batchScan)) case _ => if (batchScan.runtimeFilters.isEmpty) { throw new UnsupportedOperationException( diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/BasicScanExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/BasicScanExecTransformer.scala index 3bdea77ef63f..7bb32df6f7cf 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/BasicScanExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/BasicScanExecTransformer.scala @@ -22,8 +22,7 @@ import io.glutenproject.extension.ValidationResult import io.glutenproject.substrait.`type`.ColumnTypeNode import io.glutenproject.substrait.{SubstraitContext, SupportFormat} import io.glutenproject.substrait.plan.PlanBuilder -import io.glutenproject.substrait.rel.ReadRelNode -import io.glutenproject.substrait.rel.RelBuilder +import io.glutenproject.substrait.rel.{ReadRelNode, RelBuilder, SplitInfo} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ @@ -34,7 +33,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.common.collect.Lists -trait BasicScanExecTransformer extends TransformSupport with SupportFormat { +import scala.collection.JavaConverters._ + +trait BasicScanExecTransformer extends LeafTransformSupport with SupportFormat { // The key of merge schema option in Parquet reader. protected val mergeSchemaOptionKey = "mergeschema" @@ -52,25 +53,25 @@ trait BasicScanExecTransformer extends TransformSupport with SupportFormat { // TODO: Remove this expensive call when CH support scan custom partition location. def getInputFilePaths: Seq[String] + def getSplitInfos: Seq[SplitInfo] = + getPartitions.map( + BackendsApiManager.getIteratorApiInstance + .genSplitInfo(_, getPartitionSchemas, fileFormat)) + def doExecuteColumnarInternal(): RDD[ColumnarBatch] = { val numOutputRows = longMetric("outputRows") val numOutputVectors = longMetric("outputVectors") val scanTime = longMetric("scanTime") val substraitContext = new SubstraitContext val transformContext = doTransform(substraitContext) - val outNames = new java.util.ArrayList[String]() - for (attr <- outputAttributes()) { - outNames.add(ConverterUtils.genColumnNameWithExprId(attr)) - } + val outNames = outputAttributes().map(ConverterUtils.genColumnNameWithExprId).asJava val planNode = PlanBuilder.makePlan(substraitContext, Lists.newArrayList(transformContext.root), outNames) - val fileFormat = ConverterUtils.getFileFormat(this) BackendsApiManager.getIteratorApiInstance.genNativeFileScanRDD( sparkContext, WholeStageTransformContext(planNode, substraitContext), - fileFormat, - getPartitions, + getSplitInfos, numOutputRows, numOutputVectors, scanTime @@ -102,14 +103,14 @@ trait BasicScanExecTransformer extends TransformSupport with SupportFormat { val typeNodes = ConverterUtils.collectAttributeTypeNodes(output) val nameList = ConverterUtils.collectAttributeNamesWithoutExprId(output) val partitionSchemas = getPartitionSchemas - val columnTypeNodes = new java.util.ArrayList[ColumnTypeNode]() - for (attr <- output) { - if (partitionSchemas.exists(_.name.equals(attr.name))) { - columnTypeNodes.add(new ColumnTypeNode(1)) - } else { - columnTypeNodes.add(new ColumnTypeNode(0)) - } - } + val columnTypeNodes = output.map { + attr => + if (partitionSchemas.exists(_.name.equals(attr.name))) { + new ColumnTypeNode(1) + } else { + new ColumnTypeNode(0) + } + }.asJava // Will put all filter expressions into an AND expression val transformer = filterExprs() .reduceLeftOption(And) diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/BatchScanExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/BatchScanExecTransformer.scala index 85012f9a15cd..1d4551afa4b2 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/BatchScanExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/BatchScanExecTransformer.scala @@ -16,16 +16,17 @@ */ package io.glutenproject.execution -import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater +import io.glutenproject.sql.shims.SparkShimLoader import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat 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.connector.catalog.Table import org.apache.spark.sql.connector.read.{InputPartition, Scan} -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.v2.{BatchScanExecShim, FileScan} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType @@ -43,8 +44,13 @@ class BatchScanExecTransformer( output: Seq[AttributeReference], @transient scan: Scan, runtimeFilters: Seq[Expression], - keyGroupedPartitioning: Option[Seq[Expression]] = None) - extends BatchScanExecShim(output, scan, runtimeFilters) + keyGroupedPartitioning: Option[Seq[Expression]] = None, + ordering: Option[Seq[SortOrder]] = None, + @transient table: Table, + commonPartitionValues: Option[Seq[(InternalRow, Int)]] = None, + applyPartialClustering: Boolean = false, + replicatePartitions: Boolean = false) + extends BatchScanExecShim(output, scan, runtimeFilters, table) with BasicScanExecTransformer { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @@ -86,8 +92,6 @@ class BatchScanExecTransformer( super.doValidateInternal() } - override def supportsColumnar(): Boolean = GlutenConfig.getConf.enableColumnarIterator - override def doExecuteColumnar(): RDD[ColumnarBatch] = { doExecuteColumnarInternal() } @@ -102,18 +106,6 @@ class BatchScanExecTransformer( override def canEqual(other: Any): Boolean = other.isInstanceOf[BatchScanExecTransformer] - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { - Seq() - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = { - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genBatchScanTransformerMetricsUpdater(metrics) @@ -133,7 +125,8 @@ class BatchScanExecTransformer( new BatchScanExecTransformer( canonicalized.output, canonicalized.scan, - canonicalized.runtimeFilters + canonicalized.runtimeFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(canonicalized) ) } } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/BroadcastBuildSideRDD.scala b/gluten-core/src/main/scala/io/glutenproject/execution/BroadcastBuildSideRDD.scala index 3681c4073be9..321d97418791 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/BroadcastBuildSideRDD.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/BroadcastBuildSideRDD.scala @@ -16,32 +16,22 @@ */ package io.glutenproject.execution -import io.glutenproject.backendsapi.BackendsApiManager -import io.glutenproject.exception.GlutenException - import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.joins.BuildSideRelation import org.apache.spark.sql.vectorized.ColumnarBatch -final private case class BroadcastBuildSideRDDPartition(index: Int) extends Partition - case class BroadcastBuildSideRDD( @transient private val sc: SparkContext, broadcasted: broadcast.Broadcast[BuildSideRelation], - broadCastContext: BroadCastHashJoinContext, - numPartitions: Int = -1) + broadCastContext: BroadCastHashJoinContext) extends RDD[ColumnarBatch](sc, Nil) { override def getPartitions: Array[Partition] = { - if (numPartitions < 0) { - throw new GlutenException(s"Invalid number of partitions: $numPartitions.") - } - Array.tabulate(numPartitions)(i => BroadcastBuildSideRDDPartition(i)) + throw new IllegalStateException("Never reach here") } override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { - BackendsApiManager.getIteratorApiInstance - .genBroadcastBuildSideIterator(split, context, broadcasted, broadCastContext) + throw new IllegalStateException("Never reach here") } } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/CoalesceExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/CoalesceExecTransformer.scala index 1e488af1ba62..5ffdaf19d3e6 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/CoalesceExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/CoalesceExecTransformer.scala @@ -16,9 +16,7 @@ */ package io.glutenproject.execution -import io.glutenproject.extension.ValidationResult -import io.glutenproject.metrics.{MetricsUpdater, NoopMetricsUpdater} -import io.glutenproject.substrait.SubstraitContext +import io.glutenproject.extension.{GlutenPlan, ValidationResult} import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD @@ -30,7 +28,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class CoalesceExecTransformer(numPartitions: Int, child: SparkPlan) extends UnaryExecNode - with TransformSupport { + with GlutenPlan { override def supportsColumnar: Boolean = true @@ -40,40 +38,26 @@ case class CoalesceExecTransformer(numPartitions: Int, child: SparkPlan) if (numPartitions == 1) SinglePartition else UnknownPartitioning(numPartitions) } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { - throw new UnsupportedOperationException(s"This operator doesn't support inputRDDs.") - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - throw new UnsupportedOperationException(s"This operator doesn't support getBuildPlans.") - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override protected def doValidateInternal(): ValidationResult = - ValidationResult.notOk(s"$nodeName has not been supported") - - override def doTransform(context: SubstraitContext): TransformContext = { - throw new UnsupportedOperationException(s"$nodeName doesn't support doTransform.") - } + ValidationResult.ok override protected def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { - throw new UnsupportedOperationException(s"$nodeName doesn't support doExecuteColumnar.") + if (numPartitions == 1 && child.executeColumnar().getNumPartitions < 1) { + // Make sure we don't output an RDD with 0 partitions, when claiming that we have a + // `SinglePartition`. + new CoalesceExecTransformer.EmptyRDDWithPartitions(sparkContext, numPartitions) + } else { + child.executeColumnar().coalesce(numPartitions, shuffle = false) + } } override protected def withNewChildInternal(newChild: SparkPlan): CoalesceExecTransformer = copy(child = newChild) - override def metricsUpdater(): MetricsUpdater = new NoopMetricsUpdater } object CoalesceExecTransformer { diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala index e1771cb3d456..0a76d53a64e8 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/ExpandExecTransformer.scala @@ -27,7 +27,6 @@ import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} 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.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution._ @@ -35,8 +34,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.Any -import java.util +import java.util.{ArrayList => JArrayList, List => JList} +import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer case class ExpandExecTransformer( @@ -44,13 +44,16 @@ case class ExpandExecTransformer( output: Seq[Attribute], child: SparkPlan) extends UnaryExecNode - with TransformSupport { + with UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genExpandTransformerMetrics(sparkContext) - val originalInputAttributes: Seq[Attribute] = child.output + @transient + override lazy val references: AttributeSet = { + AttributeSet.fromAttributeSets(projections.flatten.map(_.references)) + } override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genExpandTransformerMetricsUpdater(metrics) @@ -59,26 +62,6 @@ case class ExpandExecTransformer( // as UNKNOWN partitioning override def outputPartitioning: Partitioning = UnknownPartitioning(0) - override def supportsColumnar: Boolean = true - - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - throw new UnsupportedOperationException(s"This operator doesn't support getBuildPlans.") - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - def getRelNode( context: SubstraitContext, projections: Seq[Seq[Expression]], @@ -97,9 +80,9 @@ case class ExpandExecTransformer( val preExprs = ArrayBuffer.empty[Expression] val selectionMaps = ArrayBuffer.empty[Seq[Int]] var preExprIndex = 0 - for (i <- 0 until projections.size) { + for (i <- projections.indices) { val selections = ArrayBuffer.empty[Int] - for (j <- 0 until projections(i).size) { + for (j <- projections(i).indices) { val proj = projections(i)(j) if (!proj.isInstanceOf[Literal]) { val exprIdx = preExprs.indexWhere(expr => expr.semanticEquals(proj)) @@ -117,14 +100,12 @@ case class ExpandExecTransformer( selectionMaps += selections } // make project - val preExprNodes = new util.ArrayList[ExpressionNode]() - preExprs.foreach { - expr => - val exprNode = ExpressionConverter - .replaceWithExpressionTransformer(expr, originalInputAttributes) - .doTransform(args) - preExprNodes.add(exprNode) - } + val preExprNodes = preExprs + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, originalInputAttributes) + .doTransform(args)) + .asJava val emitStartIndex = originalInputAttributes.size val inputRel = if (!validation) { @@ -147,10 +128,10 @@ case class ExpandExecTransformer( } // make expand - val projectSetExprNodes = new util.ArrayList[util.ArrayList[ExpressionNode]]() - for (i <- 0 until projections.size) { - val projectExprNodes = new util.ArrayList[ExpressionNode]() - for (j <- 0 until projections(i).size) { + val projectSetExprNodes = new JArrayList[JList[ExpressionNode]]() + for (i <- projections.indices) { + val projectExprNodes = new JArrayList[ExpressionNode]() + for (j <- projections(i).indices) { val projectExprNode = projections(i)(j) match { case l: Literal => LiteralTransformer(l).doTransform(args) @@ -164,10 +145,10 @@ case class ExpandExecTransformer( } RelBuilder.makeExpandRel(inputRel, projectSetExprNodes, context, operatorId) } else { - val projectSetExprNodes = new util.ArrayList[util.ArrayList[ExpressionNode]]() + val projectSetExprNodes = new JArrayList[JList[ExpressionNode]]() projections.foreach { projectSet => - val projectExprNodes = new util.ArrayList[ExpressionNode]() + val projectExprNodes = new JArrayList[ExpressionNode]() projectSet.foreach { project => val projectExprNode = ExpressionConverter @@ -239,11 +220,7 @@ case class ExpandExecTransformer( } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) ( getRelNode(context, projections, child.output, operatorId, readRel, validation = false), child.output) @@ -252,9 +229,6 @@ case class ExpandExecTransformer( TransformContext(inputAttributes, output, currRel) } - override protected def doExecute(): RDD[InternalRow] = - throw new UnsupportedOperationException("doExecute is not supported in ColumnarExpandExec.") - override protected def doExecuteColumnar(): RDD[ColumnarBatch] = { throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala index f6cecc995f7f..822f98c47986 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/FileSourceScanExecTransformer.scala @@ -16,29 +16,26 @@ */ package io.glutenproject.execution -import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression.ConverterUtils import io.glutenproject.extension.ValidationResult -import io.glutenproject.metrics.{GlutenTimeMetric, MetricsUpdater} +import io.glutenproject.metrics.MetricsUpdater import io.glutenproject.substrait.SubstraitContext import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat import io.glutenproject.substrait.rel.ReadRelNode import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BoundReference, DynamicPruningExpression, Expression, PlanExpression, Predicate} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, PlanExpression} import org.apache.spark.sql.connector.read.InputPartition -import org.apache.spark.sql.execution.{FileSourceScanExecShim, InSubqueryExec, ScalarSubquery, SparkPlan, SQLExecution} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.FileSourceScanExecShim +import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.collection.BitSet -import java.util.concurrent.TimeUnit.NANOSECONDS - -import scala.collection.{mutable, JavaConverters} +import scala.collection.JavaConverters class FileSourceScanExecTransformer( @transient relation: HadoopFsRelation, @@ -65,10 +62,10 @@ class FileSourceScanExecTransformer( // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics: Map[String, SQLMetric] = BackendsApiManager.getMetricsApiInstance - .genFileSourceScanTransformerMetrics(sparkContext) ++ staticMetrics + .genFileSourceScanTransformerMetrics(sparkContext) ++ staticMetricsAlias /** SQL metrics generated only for scans using dynamic partition pruning. */ - private lazy val staticMetrics = + private lazy val staticMetricsAlias = if (partitionFilters.exists(FileSourceScanExecTransformer.isDynamicPruningFilter)) { Map( "staticFilesNum" -> SQLMetrics.createMetric(sparkContext, "static number of files read"), @@ -78,14 +75,6 @@ class FileSourceScanExecTransformer( Map.empty[String, SQLMetric] } - override lazy val supportsColumnar: Boolean = { - /* - relation.fileFormat - .supportBatch(relation.sparkSession, schema) && GlutenConfig.getConf.enableColumnarIterator - */ - GlutenConfig.getConf.enableColumnarIterator - } - override def filterExprs(): Seq[Expression] = dataFilters override def outputAttributes(): Seq[Attribute] = output @@ -95,6 +84,7 @@ class FileSourceScanExecTransformer( relation, dynamicallySelectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) @@ -117,25 +107,7 @@ class FileSourceScanExecTransformer( override def hashCode(): Int = super.hashCode() - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { - Seq() - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = { - this - } - override protected def doValidateInternal(): ValidationResult = { - // Bucketing table has `bucketId` in filename, should apply this in backends - // TODO Support bucketed scan - if (bucketedScan && !BackendsApiManager.getSettings.supportBucketScan()) { - throw new UnsupportedOperationException("Bucketed scan is unsupported for now.") - } - if (hasMetadataColumns) { return ValidationResult.notOk(s"Unsupported metadataColumns scan in native.") } @@ -156,96 +128,6 @@ class FileSourceScanExecTransformer( override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genFileSourceScanTransformerMetricsUpdater(metrics) - // The codes below are copied from FileSourceScanExec in Spark, - // all of them are private. - protected lazy val driverMetrics: mutable.HashMap[String, Long] = mutable.HashMap.empty - - /** - * Send the driver-side metrics. Before calling this function, selectedPartitions has been - * initialized. See SPARK-26327 for more details. - */ - protected def sendDriverMetrics(): Unit = { - driverMetrics.foreach(e => metrics(e._1).add(e._2)) - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates( - sparkContext, - executionId, - metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) - } - - protected def setFilesNumAndSizeMetric( - partitions: Seq[PartitionDirectory], - static: Boolean): Unit = { - val filesNum = partitions.map(_.files.size.toLong).sum - val filesSize = partitions.map(_.files.map(_.getLen).sum).sum - if (!static || !partitionFilters.exists(FileSourceScanExecTransformer.isDynamicPruningFilter)) { - driverMetrics("numFiles") = filesNum - driverMetrics("filesSize") = filesSize - } else { - driverMetrics("staticFilesNum") = filesNum - driverMetrics("staticFilesSize") = filesSize - } - if (relation.partitionSchema.nonEmpty) { - driverMetrics("numPartitions") = partitions.length - } - } - - @transient override lazy val selectedPartitions: Array[PartitionDirectory] = { - val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) - GlutenTimeMetric.withNanoTime { - val ret = - relation.location.listFiles( - partitionFilters.filterNot(FileSourceScanExecTransformer.isDynamicPruningFilter), - dataFilters) - setFilesNumAndSizeMetric(ret, static = true) - ret - }(t => driverMetrics("metadataTime") = NANOSECONDS.toMillis(t + optimizerMetadataTimeNs)) - }.toArray - - // We can only determine the actual partitions at runtime when a dynamic partition filter is - // present. This is because such a filter relies on information that is only available at run - // time (for instance the keys used in the other side of a join). - @transient lazy val dynamicallySelectedPartitions: Array[PartitionDirectory] = { - val dynamicPartitionFilters = - partitionFilters.filter(FileSourceScanExecTransformer.isDynamicPruningFilter) - val selected = if (dynamicPartitionFilters.nonEmpty) { - // When it includes some DynamicPruningExpression, - // it needs to execute InSubqueryExec first, - // because doTransform path can't execute 'doExecuteColumnar' which will - // execute prepare subquery first. - dynamicPartitionFilters.foreach { - case DynamicPruningExpression(inSubquery: InSubqueryExec) => - executeInSubqueryForDynamicPruningExpression(inSubquery) - case e: Expression => - e.foreach { - case s: ScalarSubquery => s.updateResult() - case _ => - } - case _ => - } - GlutenTimeMetric.withMillisTime { - // call the file index for the files matching all filters except dynamic partition filters - val predicate = dynamicPartitionFilters.reduce(And) - val partitionColumns = relation.partitionSchema - val boundPredicate = Predicate.create( - predicate.transform { - case a: AttributeReference => - val index = partitionColumns.indexWhere(a.name == _.name) - BoundReference(index, partitionColumns(index).dataType, nullable = true) - }, - Nil - ) - val ret = selectedPartitions.filter(p => boundPredicate.eval(p.values)) - setFilesNumAndSizeMetric(ret, static = false) - ret - }(t => driverMetrics("pruningTime") = t) - } else { - selectedPartitions - } - sendDriverMetrics() - selected - } - override val nodeNamePrefix: String = "NativeFile" override val nodeName: String = { @@ -281,6 +163,7 @@ class FileSourceScanExecTransformer( case "DwrfFileFormat" => ReadFileFormat.DwrfReadFormat case "DeltaMergeTreeFileFormat" => ReadFileFormat.MergeTreeReadFormat case "CSVFileFormat" => ReadFileFormat.TextReadFormat + case "DeltaParquetFileFormat" => ReadFileFormat.ParquetReadFormat case _ => ReadFileFormat.UnknownFormat } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/GenerateExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/GenerateExecTransformer.scala index b27d76f7c1b5..9a1f6747c698 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/GenerateExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/GenerateExecTransformer.scala @@ -21,21 +21,18 @@ import io.glutenproject.exception.GlutenException import io.glutenproject.expression.{ConverterUtils, ExpressionConverter, ExpressionTransformer} import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.SubstraitContext import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} -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.execution.{SparkPlan, UnaryExecNode} -import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.sql.execution.SparkPlan import com.google.protobuf.Any -import java.util +import java.util.{ArrayList => JArrayList, List => JList} import scala.collection.JavaConverters._ @@ -47,8 +44,7 @@ case class GenerateExecTransformer( outer: Boolean, generatorOutput: Seq[Attribute], child: SparkPlan) - extends UnaryExecNode - with TransformSupport { + extends UnaryTransformSupport { @transient override lazy val metrics = @@ -58,37 +54,9 @@ case class GenerateExecTransformer( override def producedAttributes: AttributeSet = AttributeSet(generatorOutput) - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException(s"GenerateExecTransformer doesn't support doExecute") - } - override protected def withNewChildInternal(newChild: SparkPlan): GenerateExecTransformer = copy(generator, requiredChildOutput, outer, generatorOutput, newChild) - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - val childPlans = c.getBuildPlans - childPlans :+ (this, null) - case _ => - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - - override def supportsColumnar: Boolean = true - override protected def doValidateInternal(): ValidationResult = { val validationResult = BackendsApiManager.getTransformerApiInstance.validateGenerator(generator, outer) @@ -137,7 +105,7 @@ case class GenerateExecTransformer( val generatorExpr = ExpressionConverter.replaceWithExpressionTransformer(generator, child.output) val generatorNode = generatorExpr.doTransform(args) - val requiredChildOutputNodes = new java.util.ArrayList[ExpressionNode] + val requiredChildOutputNodes = new JArrayList[ExpressionNode] for (target <- requiredChildOutput) { val found = child.output.zipWithIndex.filter(_._1.name == target.name) if (found.nonEmpty) { @@ -151,19 +119,13 @@ case class GenerateExecTransformer( val inputRel = if (childCtx != null) { childCtx.root } else { - val attrList = new java.util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) readRel } val projRel = - if ( - BackendsApiManager.getSettings.insertPostProjectForGenerate() && needsProjection(generator) - ) { + if (BackendsApiManager.getSettings.insertPostProjectForGenerate()) { // need to insert one projection node for velox backend - val projectExpressions = new util.ArrayList[ExpressionNode]() + val projectExpressions = new JArrayList[ExpressionNode]() val childOutputNodes = child.output.indices .map(i => ExpressionBuilder.makeSelection(i).asInstanceOf[ExpressionNode]) .asJava @@ -196,26 +158,20 @@ case class GenerateExecTransformer( TransformContext(child.output, output, relNode) } - def needsProjection(generator: Generator): Boolean = { - !generator.asInstanceOf[Explode].child.isInstanceOf[AttributeReference] - } - def getRelNode( context: SubstraitContext, operatorId: Long, inputAttributes: Seq[Attribute], input: RelNode, generator: ExpressionNode, - childOutput: util.ArrayList[ExpressionNode], + childOutput: JList[ExpressionNode], validation: Boolean): RelNode = { if (!validation) { RelBuilder.makeGenerateRel(input, generator, childOutput, context, operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- inputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = + inputAttributes.map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)).asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeGenerateRel(input, generator, childOutput, extensionNode, context, operatorId) diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/GlutenWholeStageColumnarRDD.scala b/gluten-core/src/main/scala/io/glutenproject/execution/GlutenWholeStageColumnarRDD.scala index a23d3d399155..11b95251afb6 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/GlutenWholeStageColumnarRDD.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/GlutenWholeStageColumnarRDD.scala @@ -21,7 +21,7 @@ import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.metrics.IMetrics import io.glutenproject.substrait.plan.PlanBuilder -import org.apache.spark.{OneToOneDependency, Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.datasources.PartitionedFile @@ -30,23 +30,22 @@ import org.apache.spark.sql.utils.OASPackageBridge.InputMetricsWrapper import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.ExecutorManager -import io.substrait.proto.Plan - -import java.io.Serializable - import scala.collection.mutable trait BaseGlutenPartition extends Partition with InputPartition { - def plan: Plan + def plan: Array[Byte] } -case class GlutenPartition(index: Int, plan: Plan, locations: Array[String] = Array.empty[String]) +case class GlutenPartition( + index: Int, + plan: Array[Byte], + locations: Array[String] = Array.empty[String]) extends BaseGlutenPartition { override def preferredLocations(): Array[String] = locations } -case class GlutenFilePartition(index: Int, files: Array[PartitionedFile], plan: Plan) +case class GlutenFilePartition(index: Int, files: Array[PartitionedFile], plan: Array[Byte]) extends BaseGlutenPartition { override def preferredLocations(): Array[String] = { // Computes total number of bytes can be retrieved from each host. @@ -76,94 +75,62 @@ case class GlutenMergeTreePartition( tablePath: String, minParts: Long, maxParts: Long, - plan: Plan = PlanBuilder.empty().toProtobuf) + plan: Array[Byte] = PlanBuilder.EMPTY_PLAN) extends BaseGlutenPartition { override def preferredLocations(): Array[String] = { Array.empty[String] } - - def copySubstraitPlan(newSubstraitPlan: Plan): GlutenMergeTreePartition = { - this.copy(plan = newSubstraitPlan) - } } case class FirstZippedPartitionsPartition( - idx: Int, + index: Int, inputPartition: InputPartition, - @transient private val rdds: Seq[RDD[_]] = Seq()) + inputColumnarRDDPartitions: Seq[Partition] = Seq.empty) extends Partition - with Serializable { - - override val index: Int = idx - var partitionValues = rdds.map(rdd => rdd.partitions(idx)) - - def partitions: Seq[Partition] = partitionValues -} class GlutenWholeStageColumnarRDD( @transient sc: SparkContext, @transient private val inputPartitions: Seq[InputPartition], - var rdds: Seq[RDD[ColumnarBatch]], + var rdds: ColumnarInputRDDsWrapper, pipelineTime: SQLMetric, updateInputMetrics: (InputMetricsWrapper) => Unit, updateNativeMetrics: IMetrics => Unit) - extends RDD[ColumnarBatch](sc, rdds.map(x => new OneToOneDependency(x))) { + extends RDD[ColumnarBatch](sc, rdds.getDependencies) { val numaBindingInfo = GlutenConfig.getConf.numaBindingInfo override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { ExecutorManager.tryTaskSet(numaBindingInfo) - - val inputPartition = castNativePartition(split) - if (rdds.isEmpty) { - BackendsApiManager.getIteratorApiInstance.genFirstStageIterator( - inputPartition, - context, - pipelineTime, - updateInputMetrics, - updateNativeMetrics) - } else { - val partitions = split.asInstanceOf[FirstZippedPartitionsPartition].partitions - val inputIterators = - (rdds.zip(partitions)).map { case (rdd, partition) => rdd.iterator(partition, context) } - BackendsApiManager.getIteratorApiInstance.genFirstStageIterator( - inputPartition, - context, - pipelineTime, - updateInputMetrics, - updateNativeMetrics, - inputIterators - ) - } + val (inputPartition, inputColumnarRDDPartitions) = castNativePartition(split) + val inputIterators = rdds.getIterators(inputColumnarRDDPartitions, context) + BackendsApiManager.getIteratorApiInstance.genFirstStageIterator( + inputPartition, + context, + pipelineTime, + updateInputMetrics, + updateNativeMetrics, + inputIterators + ) } - private def castNativePartition(split: Partition): BaseGlutenPartition = split match { - case FirstZippedPartitionsPartition(_, p: BaseGlutenPartition, _) => p - case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split") + private def castNativePartition(split: Partition): (BaseGlutenPartition, Seq[Partition]) = { + split match { + case FirstZippedPartitionsPartition(_, g: BaseGlutenPartition, p) => (g, p) + case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split") + } } override def getPreferredLocations(split: Partition): Seq[String] = { - castPartition(split).inputPartition.preferredLocations() - } - - private def castPartition(split: Partition): FirstZippedPartitionsPartition = split match { - case p: FirstZippedPartitionsPartition => p - case _ => throw new SparkException(s"[BUG] Not a NativeSubstraitPartition: $split") + castNativePartition(split)._1.preferredLocations() } override protected def getPartitions: Array[Partition] = { - if (rdds.isEmpty) { - inputPartitions.zipWithIndex.map { - case (inputPartition, index) => FirstZippedPartitionsPartition(index, inputPartition) - }.toArray - } else { - val numParts = inputPartitions.size - if (!rdds.forall(rdd => rdd.partitions.length == numParts)) { - throw new IllegalArgumentException( - s"Can't zip RDDs with unequal numbers of partitions: ${rdds.map(_.partitions.length)}") - } - Array.tabulate[Partition](numParts) { - i => FirstZippedPartitionsPartition(i, inputPartitions(i), rdds) - } + Array.tabulate[Partition](inputPartitions.size) { + i => FirstZippedPartitionsPartition(i, inputPartitions(i), rdds.getPartitions(i)) } } + + override protected def clearDependencies(): Unit = { + super.clearDependencies() + rdds = null + } } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/HashAggregateExecBaseTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/HashAggregateExecBaseTransformer.scala index 0e94784f328a..25298c53f710 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/HashAggregateExecBaseTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/HashAggregateExecBaseTransformer.scala @@ -16,12 +16,11 @@ */ package io.glutenproject.execution -import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression._ import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.{AggregationParams, SubstraitContext} import io.glutenproject.substrait.expression.{AggregateFunctionNode, ExpressionBuilder, ExpressionNode} import io.glutenproject.substrait.extensions.ExtensionBuilder @@ -35,14 +34,13 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.aggregate._ import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.sketch.BloomFilter import com.google.protobuf.Any -import java.util +import java.util.{ArrayList => JArrayList, List => JList} +import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer -import scala.util.control.Breaks.{break, breakable} /** Columnar Based HashAggregateExec. */ abstract class HashAggregateExecBaseTransformer( @@ -54,7 +52,7 @@ abstract class HashAggregateExecBaseTransformer( resultExpressions: Seq[NamedExpression], child: SparkPlan) extends BaseAggregateExec - with TransformSupport { + with UnaryTransformSupport { override lazy val allAttributes: AttributeSeq = child.output ++ aggregateBufferAttributes ++ aggregateAttributes ++ @@ -64,8 +62,6 @@ abstract class HashAggregateExecBaseTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genHashAggregateTransformerMetrics(sparkContext) - val sparkConf = sparkContext.getConf - // The direct outputs of Aggregation. protected lazy val allAggregateResultAttributes: List[Attribute] = { val groupingAttributes = groupingExpressions.map( @@ -77,33 +73,10 @@ abstract class HashAggregateExecBaseTransformer( aggregateAttributes) } - override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator - override def doExecuteColumnar(): RDD[ColumnarBatch] = { throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - c.getBuildPlans - case _ => - Seq() - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genHashAggregateTransformerMetricsUpdater(metrics) @@ -130,10 +103,10 @@ abstract class HashAggregateExecBaseTransformer( case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | StringType | TimestampType | DateType | BinaryType => true - case d: DecimalType => true - case a: ArrayType => true - case n: NullType => true - case other => false + case _: DecimalType => true + case _: ArrayType => true + case _: NullType => true + case _ => false } } @@ -172,11 +145,7 @@ abstract class HashAggregateExecBaseTransformer( // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. aggParams.isReadRel = true - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) (getAggRel(context, operatorId, aggParams, readRel), child.output, output) } TransformContext(inputAttributes, outputAttributes, relNode) @@ -185,79 +154,45 @@ abstract class HashAggregateExecBaseTransformer( // Members declared in org.apache.spark.sql.execution.AliasAwareOutputPartitioning override protected def outputExpressions: Seq[NamedExpression] = resultExpressions - // Members declared in org.apache.spark.sql.execution.SparkPlan - override protected def doExecute() - : org.apache.spark.rdd.RDD[org.apache.spark.sql.catalyst.InternalRow] = - throw new UnsupportedOperationException() - + // Check if Pre-Projection is needed before the Aggregation. protected def needsPreProjection: Boolean = { - var needsProjection = false - breakable { - for (expr <- groupingExpressions) { - if (!expr.isInstanceOf[Attribute]) { - needsProjection = true - break - } - } - } - breakable { - for (expr <- aggregateExpressions) { - if ( - expr.filter.isDefined && !expr.filter.get.isInstanceOf[Attribute] && - !expr.filter.get.isInstanceOf[Literal] - ) { - needsProjection = true - break + groupingExpressions.exists { + case _: Attribute => false + case _ => true + } || aggregateExpressions.exists { + expr => + expr.filter match { + case None | Some(_: Attribute) | Some(_: Literal) => + case _ => return true } expr.mode match { case Partial => - for (aggChild <- expr.aggregateFunction.children) { - if (!aggChild.isInstanceOf[Attribute] && !aggChild.isInstanceOf[Literal]) { - needsProjection = true - break - } + expr.aggregateFunction.children.exists { + case _: Attribute | _: Literal => false + case _ => true } // No need to consider pre-projection for PartialMerge and Final Agg. - case _ => + case _ => false } - } } - needsProjection } + // Check if Post-Projection is needed after the Aggregation. protected def needsPostProjection(aggOutAttributes: List[Attribute]): Boolean = { - // Check if Post-Projection is needed after the Aggregation. - var needsProjection = false // If the result expressions has different size with output attribute, // post-projection is needed. - if (resultExpressions.size != aggOutAttributes.size) { - needsProjection = true - } else { - // Compare each item in result expressions and output attributes. - breakable { - for (exprIdx <- resultExpressions.indices) { - resultExpressions(exprIdx) match { - case exprAttr: Attribute => - val resAttr = aggOutAttributes(exprIdx) - // If the result attribute and result expression has different name or type, - // post-projection is needed. - if ( - exprAttr.name != resAttr.name || - exprAttr.dataType != resAttr.dataType - ) { - needsProjection = true - break - } - case _ => - // If result expression is not instance of Attribute, - // post-projection is needed. - needsProjection = true - break - } - } - } + resultExpressions.size != aggOutAttributes.size || + // Compare each item in result expressions and output attributes. + resultExpressions.zip(aggOutAttributes).exists { + case (exprAttr: Attribute, resAttr) => + // If the result attribute and result expression has different name or type, + // post-projection is needed. + exprAttr.name != resAttr.name || exprAttr.dataType != resAttr.dataType + case _ => + // If result expression is not instance of Attribute, + // post-projection is needed. + true } - needsProjection } protected def getAggRelWithPreProjection( @@ -314,22 +249,20 @@ abstract class HashAggregateExecBaseTransformer( }) // Create the expression nodes needed by Project node. - val preExprNodes = new util.ArrayList[ExpressionNode]() - for (expr <- preExpressions) { - preExprNodes.add( + val preExprNodes = preExpressions + .map( ExpressionConverter - .replaceWithExpressionTransformer(expr, originalInputAttributes) + .replaceWithExpressionTransformer(_, originalInputAttributes) .doTransform(args)) - } + .asJava val emitStartIndex = originalInputAttributes.size val inputRel = if (!validation) { RelBuilder.makeProjectRel(input, preExprNodes, context, operatorId, emitStartIndex) } else { // Use a extension node to send the input types through Substrait plan for a validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeProjectRel( @@ -352,7 +285,7 @@ abstract class HashAggregateExecBaseTransformer( filterSelections: Seq[Int], inputRel: RelNode, operatorId: Long): RelNode = { - val groupingList = new util.ArrayList[ExpressionNode]() + val groupingList = new JArrayList[ExpressionNode]() var colIdx = 0 while (colIdx < groupingExpressions.size) { val groupingExpr: ExpressionNode = ExpressionBuilder.makeSelection(selections(colIdx)) @@ -361,11 +294,11 @@ abstract class HashAggregateExecBaseTransformer( } // Create Aggregation functions. - val aggregateFunctionList = new util.ArrayList[AggregateFunctionNode]() + val aggregateFunctionList = new JArrayList[AggregateFunctionNode]() aggregateExpressions.foreach( aggExpr => { val aggregateFunc = aggExpr.aggregateFunction - val childrenNodeList = new util.ArrayList[ExpressionNode]() + val childrenNodeList = new JArrayList[ExpressionNode]() val childrenNodes = aggregateFunc.children.toList.map( _ => { val aggExpr = ExpressionBuilder.makeSelection(selections(colIdx)) @@ -383,7 +316,7 @@ abstract class HashAggregateExecBaseTransformer( aggregateFunctionList) }) - val aggFilterList = new util.ArrayList[ExpressionNode]() + val aggFilterList = new JArrayList[ExpressionNode]() aggregateExpressions.foreach( aggExpr => { if (aggExpr.filter.isDefined) { @@ -407,9 +340,9 @@ abstract class HashAggregateExecBaseTransformer( protected def addFunctionNode( args: java.lang.Object, aggregateFunction: AggregateFunction, - childrenNodeList: util.ArrayList[ExpressionNode], + childrenNodeList: JList[ExpressionNode], aggregateMode: AggregateMode, - aggregateNodeList: util.ArrayList[AggregateFunctionNode]): Unit = { + aggregateNodeList: JList[AggregateFunctionNode]): Unit = { aggregateNodeList.add( ExpressionBuilder.makeAggregateFunction( AggregateFunctionsBuilder.create(args, aggregateFunction), @@ -427,23 +360,20 @@ abstract class HashAggregateExecBaseTransformer( val args = context.registeredFunction // Will add an projection after Agg. - val resExprNodes = new util.ArrayList[ExpressionNode]() - resultExpressions.foreach( - expr => { - resExprNodes.add( - ExpressionConverter - .replaceWithExpressionTransformer(expr, allAggregateResultAttributes) - .doTransform(args)) - }) + val resExprNodes = resultExpressions + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, allAggregateResultAttributes) + .doTransform(args)) + .asJava val emitStartIndex = allAggregateResultAttributes.size if (!validation) { RelBuilder.makeProjectRel(aggRel, resExprNodes, context, operatorId, emitStartIndex) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- allAggregateResultAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = allAggregateResultAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeProjectRel( @@ -460,7 +390,7 @@ abstract class HashAggregateExecBaseTransformer( protected def getAttrForAggregateExprs( aggregateExpressions: Seq[AggregateExpression], aggregateAttributeList: Seq[Attribute]): List[Attribute] = { - var aggregateAttr = new ListBuffer[Attribute]() + val aggregateAttr = new ListBuffer[Attribute]() val size = aggregateExpressions.size var resIndex = 0 for (expIdx <- 0 until size) { @@ -478,166 +408,63 @@ abstract class HashAggregateExecBaseTransformer( var resIndex = index val mode = exp.mode val aggregateFunc = exp.aggregateFunction - aggregateFunc match { - case _: Average | _: First | _: Last => - mode match { - case Partial | PartialMerge => - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += 2 - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case Sum(_, _) => - mode match { - case Partial | PartialMerge => - val sum = aggregateFunc.asInstanceOf[Sum] - val aggBufferAttr = sum.inputAggBufferAttributes - if (aggBufferAttr.size == 2) { - // decimal sum check sum.resultType - aggregateAttr += ConverterUtils.getAttrFromExpr(aggBufferAttr.head) - val isEmptyAttr = ConverterUtils.getAttrFromExpr(aggBufferAttr(1)) - aggregateAttr += isEmptyAttr - resIndex += 2 - } else { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr.head) - aggregateAttr += attr - resIndex += 1 - } - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case Count(_) => - mode match { - case Partial | PartialMerge => - val count = aggregateFunc.asInstanceOf[Count] - val aggBufferAttr = count.inputAggBufferAttributes - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr.head) - aggregateAttr += attr - resIndex += 1 - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case _: Max | _: Min | _: BitAndAgg | _: BitOrAgg | _: BitXorAgg => - mode match { - case Partial | PartialMerge => - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - assert( - aggBufferAttr.size == 1, - s"Aggregate function $aggregateFunc expects one buffer attribute.") - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr.head) + // First handle the custom aggregate functions + if ( + ExpressionMappings.expressionExtensionTransformer.extensionExpressionsMapping.contains( + aggregateFunc.getClass) + ) { + ExpressionMappings.expressionExtensionTransformer + .getAttrsIndexForExtensionAggregateExpr( + aggregateFunc, + mode, + exp, + aggregateAttributeList, + aggregateAttr, + index) + } else { + if (!checkAggFuncModeSupport(aggregateFunc, mode)) { + throw new UnsupportedOperationException( + s"Unsupported aggregate mode: $mode for ${aggregateFunc.prettyName}") + } + mode match { + case Partial | PartialMerge => + val aggBufferAttr = aggregateFunc.inputAggBufferAttributes + for (index <- aggBufferAttr.indices) { + val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) aggregateAttr += attr - resIndex += 1 - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case _: Corr => - mode match { - case Partial | PartialMerge => - val expectedBufferSize = 6 - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - assert( - aggBufferAttr.size == expectedBufferSize, - s"Aggregate function $aggregateFunc" + - s" expects $expectedBufferSize buffer attribute.") - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += expectedBufferSize - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case _: CovPopulation | _: CovSample => - mode match { - case Partial | PartialMerge => - val expectedBufferSize = 4 - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - assert( - aggBufferAttr.size == expectedBufferSize, - s"Aggregate function $aggregateFunc" + - s" expects $expectedBufferSize buffer attributes.") - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += expectedBufferSize - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") - } - case _: StddevSamp | _: StddevPop | _: VarianceSamp | _: VariancePop => + } + resIndex += aggBufferAttr.size + resIndex + case Final => + aggregateAttr += aggregateAttributeList(resIndex) + resIndex += 1 + resIndex + case other => + throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") + } + } + } + + protected def checkAggFuncModeSupport( + aggFunc: AggregateFunction, + mode: AggregateMode): Boolean = { + aggFunc match { + case _: CollectList | _: CollectSet => mode match { - case Partial | PartialMerge => - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += 3 - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") + case Partial | Final => true + case _ => false } case bloom if bloom.getClass.getSimpleName.equals("BloomFilterAggregate") => - // for spark33 mode match { - case Partial => - val bloom = aggregateFunc.asInstanceOf[TypedImperativeAggregate[BloomFilter]] - val aggBufferAttr = bloom.inputAggBufferAttributes - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += aggBufferAttr.size - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") + case Partial | Final => true + case _ => false } - case _: CollectList | _: CollectSet => + case _ => mode match { - case Partial => - val aggBufferAttr = aggregateFunc.inputAggBufferAttributes - for (index <- aggBufferAttr.indices) { - val attr = ConverterUtils.getAttrFromExpr(aggBufferAttr(index)) - aggregateAttr += attr - } - resIndex += aggBufferAttr.size - case Final => - aggregateAttr += aggregateAttributeList(resIndex) - resIndex += 1 - case other => - throw new UnsupportedOperationException(s"Unsupported aggregate mode: $other.") + case Partial | PartialMerge | Final => true + case _ => false } - case other => - throw new UnsupportedOperationException( - s"Unsupported aggregate function in getAttrForAggregateExpr") } - resIndex } protected def modeToKeyWord(aggregateMode: AggregateMode): String = { @@ -658,19 +485,17 @@ abstract class HashAggregateExecBaseTransformer( validation: Boolean): RelNode = { val args = context.registeredFunction // Get the grouping nodes. - val groupingList = new util.ArrayList[ExpressionNode]() - groupingExpressions.foreach( - expr => { - // Use 'child.output' as based Seq[Attribute], the originalInputAttributes - // may be different for each backend. - val exprNode = ExpressionConverter - .replaceWithExpressionTransformer(expr, child.output) - .doTransform(args) - groupingList.add(exprNode) - }) + // Use 'child.output' as based Seq[Attribute], the originalInputAttributes + // may be different for each backend. + val groupingList = groupingExpressions + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, child.output) + .doTransform(args)) + .asJava // Get the aggregate function nodes. - val aggFilterList = new util.ArrayList[ExpressionNode]() - val aggregateFunctionList = new util.ArrayList[AggregateFunctionNode]() + val aggFilterList = new JArrayList[ExpressionNode]() + val aggregateFunctionList = new JArrayList[AggregateFunctionNode]() aggregateExpressions.foreach( aggExpr => { if (aggExpr.filter.isDefined) { @@ -683,7 +508,6 @@ abstract class HashAggregateExecBaseTransformer( aggFilterList.add(null) } val aggregateFunc = aggExpr.aggregateFunction - val childrenNodeList = new util.ArrayList[ExpressionNode]() val childrenNodes = aggExpr.mode match { case Partial => aggregateFunc.children.toList.map( @@ -702,10 +526,12 @@ abstract class HashAggregateExecBaseTransformer( case other => throw new UnsupportedOperationException(s"$other not supported.") } - for (node <- childrenNodes) { - childrenNodeList.add(node) - } - addFunctionNode(args, aggregateFunc, childrenNodeList, aggExpr.mode, aggregateFunctionList) + addFunctionNode( + args, + aggregateFunc, + childrenNodes.asJava, + aggExpr.mode, + aggregateFunctionList) }) if (!validation) { RelBuilder.makeAggregateRel( @@ -717,10 +543,9 @@ abstract class HashAggregateExecBaseTransformer( operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeAggregateRel( @@ -739,19 +564,5 @@ abstract class HashAggregateExecBaseTransformer( operatorId: Long, aggParams: AggregationParams, input: RelNode = null, - validation: Boolean = false): RelNode = { - val originalInputAttributes = child.output - val aggRel = if (needsPreProjection) { - getAggRelWithPreProjection(context, originalInputAttributes, operatorId, input, validation) - } else { - getAggRelWithoutPreProjection(context, originalInputAttributes, operatorId, input, validation) - } - // Will check if post-projection is needed. If yes, a ProjectRel will be added after the - // AggregateRel. - if (!needsPostProjection(allAggregateResultAttributes)) { - aggRel - } else { - applyPostProjection(context, aggRel, operatorId, validation) - } - } + validation: Boolean = false): RelNode } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala index 24d7087e7ad1..67bff42ac07d 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/HashJoinExecTransformer.scala @@ -28,7 +28,6 @@ import io.glutenproject.substrait.rel.{RelBuilder, RelNode} import io.glutenproject.utils.SubstraitUtil 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.optimizer.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.catalyst.plans._ @@ -44,10 +43,9 @@ import com.google.protobuf.{Any, StringValue} import io.substrait.proto.JoinRel import java.lang.{Long => JLong} -import java.util.{ArrayList => JArrayList, HashMap => JHashMap} +import java.util.{Map => JMap} import scala.collection.JavaConverters._ -import scala.util.control.Breaks.{break, breakable} trait ColumnarShuffledJoin extends BaseJoinExec { def isSkewJoin: Boolean @@ -192,23 +190,6 @@ trait HashJoinLikeExecTransformer } } - override def supportsColumnar: Boolean = true - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = buildPlan match { - case c: TransformSupport => - val childPlans = c.getBuildPlans - childPlans :+ (this, null) - case _ => - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = streamedPlan match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override protected def doValidateInternal(): ValidationResult = { val substraitContext = new SubstraitContext // Firstly, need to check if the Substrait plan for this operator can be successfully generated. @@ -238,27 +219,29 @@ trait HashJoinLikeExecTransformer override def doTransform(substraitContext: SubstraitContext): TransformContext = { - def transformAndGetOutput(plan: SparkPlan): (RelNode, Seq[Attribute], Boolean) = { + def transformAndGetOutput(plan: SparkPlan): (RelNode, Seq[Attribute], Boolean, JLong) = { plan match { case p: TransformSupport => val transformContext = p.doTransform(substraitContext) - (transformContext.root, transformContext.outputAttributes, false) + (transformContext.root, transformContext.outputAttributes, false, -1L) case _ => val readRel = RelBuilder.makeReadRel( - new JArrayList[Attribute](plan.output.asJava), + plan.output.asJava, substraitContext, -1 ) /* A special handling in Join to delay the rel registration. */ - (readRel, plan.output, true) + // Make sure create a new read relId for the stream side first + // before the one of the build side, when there is no shuffle on the build side + (readRel, plan.output, true, substraitContext.nextRelId()) } } val joinParams = new JoinParams - val (inputStreamedRelNode, inputStreamedOutput, isStreamedReadRel) = + val (inputStreamedRelNode, inputStreamedOutput, isStreamedReadRel, streamdReadRelId) = transformAndGetOutput(streamedPlan) joinParams.isStreamedReadRel = isStreamedReadRel - val (inputBuildRelNode, inputBuildOutput, isBuildReadRel) = + val (inputBuildRelNode, inputBuildOutput, isBuildReadRel, buildReadRelId) = transformAndGetOutput(buildPlan) joinParams.isBuildReadRel = isBuildReadRel @@ -267,10 +250,10 @@ trait HashJoinLikeExecTransformer // Register the ReadRel to correct operator Id. if (joinParams.isStreamedReadRel) { - substraitContext.registerRelToOperator(operatorId) + substraitContext.registerRelToOperator(operatorId, streamdReadRelId) } if (joinParams.isBuildReadRel) { - substraitContext.registerRelToOperator(operatorId) + substraitContext.registerRelToOperator(operatorId, buildReadRelId) } if (JoinUtils.preProjectionNeeded(streamedKeyExprs)) { @@ -346,11 +329,6 @@ trait HashJoinLikeExecTransformer def genJoinParameters(): (Int, Int, String) = { (0, 0, "") } - - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException( - s"${this.getClass.getSimpleName} doesn't support doExecute") - } } object HashJoinLikeExecTransformer { @@ -359,7 +337,7 @@ object HashJoinLikeExecTransformer { leftType: DataType, rightNode: ExpressionNode, rightType: DataType, - functionMap: JHashMap[String, JLong]): ExpressionNode = { + functionMap: JMap[String, JLong]): ExpressionNode = { val functionId = ExpressionBuilder.newScalarFunction( functionMap, ConverterUtils.makeFuncName(ExpressionNames.EQUAL, Seq(leftType, rightType))) @@ -373,7 +351,7 @@ object HashJoinLikeExecTransformer { def makeAndExpression( leftNode: ExpressionNode, rightNode: ExpressionNode, - functionMap: JHashMap[String, JLong]): ExpressionNode = { + functionMap: JMap[String, JLong]): ExpressionNode = { val functionId = ExpressionBuilder.newScalarFunction( functionMap, ConverterUtils.makeFuncName(ExpressionNames.AND, Seq(BooleanType, BooleanType))) @@ -443,39 +421,6 @@ abstract class BroadcastHashJoinExecTransformer( BackendsApiManager.getBroadcastApiInstance .collectExecutionBroadcastHashTableId(executionId, context.buildHashTableId) - val buildRDD = if (streamedRDD.isEmpty) { - // Stream plan itself contains scan and has no input rdd, - // so the number of partitions cannot be decided here. - BroadcastBuildSideRDD(sparkContext, broadcast, context) - } else { - // Try to get the number of partitions from a non-broadcast RDD. - val nonBroadcastRDD = streamedRDD.find(rdd => !rdd.isInstanceOf[BroadcastBuildSideRDD]) - if (nonBroadcastRDD.isDefined) { - BroadcastBuildSideRDD( - sparkContext, - broadcast, - context, - nonBroadcastRDD.orNull.getNumPartitions) - } else { - // When all stream RDDs are broadcast RDD, the number of partitions can be undecided - // because stream plan may contain scan. - var partitions = -1 - breakable { - for (rdd <- streamedRDD) { - try { - partitions = rdd.getNumPartitions - break - } catch { - case _: Throwable => - // The partitions of this RDD is not decided yet. - } - } - } - // If all the stream RDDs are broadcast RDD, - // the number of partitions will be decided later in whole stage transformer. - BroadcastBuildSideRDD(sparkContext, broadcast, context, partitions) - } - } - streamedRDD :+ buildRDD + streamedRDD :+ BroadcastBuildSideRDD(sparkContext, broadcast, context) } } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/JoinUtils.scala b/gluten-core/src/main/scala/io/glutenproject/execution/JoinUtils.scala index 0ca1654a41d5..4f256debb058 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/JoinUtils.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/JoinUtils.scala @@ -17,7 +17,7 @@ package io.glutenproject.execution import io.glutenproject.expression.{AttributeReferenceTransformer, ConverterUtils, ExpressionConverter} -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.SubstraitContext import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} import io.glutenproject.substrait.extensions.{AdvancedExtensionNode, ExtensionBuilder} @@ -30,8 +30,6 @@ import org.apache.spark.sql.types.DataType import com.google.protobuf.Any import io.substrait.proto.JoinRel -import java.util - import scala.collection.JavaConverters._ object JoinUtils { @@ -43,7 +41,7 @@ object JoinUtils { // is also used in execution phase. In this case an empty typeUrlPrefix need to be passed, // so that it can be correctly parsed into json string on the cpp side. Any.pack( - TypeBuilder.makeStruct(false, new util.ArrayList[TypeNode](inputTypeNodes.asJava)).toProtobuf, + TypeBuilder.makeStruct(false, inputTypeNodes.asJava).toProtobuf, /* typeUrlPrefix */ "") } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/LimitTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/LimitTransformer.scala index b5840dbf152a..961e0c95201f 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/LimitTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/LimitTransformer.scala @@ -20,62 +20,32 @@ import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression.ConverterUtils import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.SubstraitContext import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.Any -import java.util +import scala.collection.JavaConverters._ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) - extends UnaryExecNode - with TransformSupport { + extends UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genLimitTransformerMetrics(sparkContext) - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - val childPlans = c.getBuildPlans - childPlans :+ (this, null) - case _ => - Seq((this, null)) - } - override protected def withNewChildInternal(newChild: SparkPlan): LimitTransformer = copy(child = newChild) - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException(s"ColumnarSortExec doesn't support doExecute") - } - override def doExecuteColumnar(): RDD[ColumnarBatch] = { throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } @@ -86,7 +56,11 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) override protected def doValidateInternal(): ValidationResult = { val context = new SubstraitContext val operatorId = context.nextOperatorId(this.nodeName) - val relNode = getRelNode(context, operatorId, offset, count, child.output, null, true) + val input = child match { + case c: TransformSupport => c.doTransform(context).root + case _ => null + } + val relNode = getRelNode(context, operatorId, offset, count, child.output, input, true) doNativeValidation(context, relNode) } @@ -101,11 +75,7 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) val relNode = if (childCtx != null) { getRelNode(context, operatorId, offset, count, child.output, childCtx.root, false) } else { - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) getRelNode(context, operatorId, offset, count, child.output, readRel, false) } TransformContext(child.output, child.output, relNode) @@ -122,10 +92,8 @@ case class LimitTransformer(child: SparkPlan, offset: Long, count: Long) if (!validation) { RelBuilder.makeFetchRel(input, offset, count, context, operatorId) } else { - val inputTypeNodes = new util.ArrayList[TypeNode]() - for (attr <- inputAttributes) { - inputTypeNodes.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodes = + inputAttributes.map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)).asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodes).toProtobuf)) RelBuilder.makeFetchRel(input, offset, count, extensionNode, context, operatorId) diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala index f6a862b5ec90..7280891cbc86 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/SortExecTransformer.scala @@ -18,7 +18,7 @@ package io.glutenproject.execution import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression.{ConverterUtils, ExpressionConverter} -import io.glutenproject.extension.{GlutenPlan, ValidationResult} +import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} import io.glutenproject.substrait.SubstraitContext @@ -27,7 +27,6 @@ import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} 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._ import org.apache.spark.sql.execution._ @@ -36,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.Any import io.substrait.proto.SortField -import java.util +import java.util.{ArrayList => JArrayList} import scala.collection.JavaConverters._ import scala.util.control.Breaks.{break, breakable} @@ -46,9 +45,7 @@ case class SortExecTransformer( global: Boolean, child: SparkPlan, testSpillFrequency: Int = 0) - extends UnaryExecNode - with TransformSupport - with GlutenPlan { + extends UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = @@ -57,10 +54,6 @@ case class SortExecTransformer( override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genSortTransformerMetricsUpdater(metrics) - val sparkConf = sparkContext.getConf - - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output override def outputOrdering: Seq[SortOrder] = sortOrder @@ -70,28 +63,6 @@ case class SortExecTransformer( override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - val childPlans = c.getBuildPlans - childPlans :+ (this, null) - case _ => - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - def getRelWithProject( context: SubstraitContext, sortOrder: Seq[SortOrder], @@ -101,13 +72,13 @@ case class SortExecTransformer( validation: Boolean): RelNode = { val args = context.registeredFunction - val sortFieldList = new util.ArrayList[SortField]() - val projectExpressions = new util.ArrayList[ExpressionNode]() - val sortExprArttributes = new util.ArrayList[AttributeReference]() + val sortFieldList = new JArrayList[SortField]() + val projectExpressions = new JArrayList[ExpressionNode]() + val sortExprAttributes = new JArrayList[AttributeReference]() val selectOrigins = - originalInputAttributes.indices.map(ExpressionBuilder.makeSelection(_)) - projectExpressions.addAll(selectOrigins.asJava) + originalInputAttributes.indices.map(ExpressionBuilder.makeSelection(_)).asJava + projectExpressions.addAll(selectOrigins) var colIdx = originalInputAttributes.size sortOrder.foreach( @@ -119,7 +90,7 @@ case class SortExecTransformer( projectExpressions.add(projectExprNode) val exprNode = ExpressionBuilder.makeSelection(colIdx) - sortExprArttributes.add(AttributeReference(s"col_$colIdx", order.child.dataType)()) + sortExprAttributes.add(AttributeReference(s"col_$colIdx", order.child.dataType)()) colIdx += 1 builder.setExpr(exprNode.toProtobuf) @@ -138,7 +109,7 @@ case class SortExecTransformer( for (attr <- originalInputAttributes) { inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) } - sortExprArttributes.forEach { + sortExprAttributes.forEach { attr => inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) } @@ -162,7 +133,7 @@ case class SortExecTransformer( inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) } - sortExprArttributes.forEach { + sortExprAttributes.forEach { attr => inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) } @@ -176,7 +147,7 @@ case class SortExecTransformer( if (!validation) { RelBuilder.makeProjectRel( sortRel, - new java.util.ArrayList[ExpressionNode](selectOrigins.asJava), + new JArrayList[ExpressionNode](selectOrigins), context, operatorId, originalInputAttributes.size + sortFieldList.size) @@ -191,7 +162,7 @@ case class SortExecTransformer( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) RelBuilder.makeProjectRel( sortRel, - new java.util.ArrayList[ExpressionNode](selectOrigins.asJava), + new JArrayList[ExpressionNode](selectOrigins), extensionNode, context, operatorId, @@ -207,9 +178,8 @@ case class SortExecTransformer( input: RelNode, validation: Boolean): RelNode = { val args = context.registeredFunction - val sortFieldList = new util.ArrayList[SortField]() - sortOrder.foreach( - order => { + val sortFieldList = sortOrder.map { + order => val builder = SortField.newBuilder() val exprNode = ExpressionConverter .replaceWithExpressionTransformer(order.child, attributeSeq = child.output) @@ -218,20 +188,18 @@ case class SortExecTransformer( builder.setDirectionValue( SortExecTransformer.transformSortDirection(order.direction.sql, order.nullOrdering.sql)) - sortFieldList.add(builder.build()) - }) + builder.build() + } if (!validation) { - RelBuilder.makeSortRel(input, sortFieldList, context, operatorId) + RelBuilder.makeSortRel(input, sortFieldList.asJava, context, operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes.map( + attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) val extensionNode = ExtensionBuilder.makeAdvancedExtension( - Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) + Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList.asJava).toProtobuf)) - RelBuilder.makeSortRel(input, sortFieldList, extensionNode, context, operatorId) + RelBuilder.makeSortRel(input, sortFieldList.asJava, extensionNode, context, operatorId) } } @@ -292,11 +260,7 @@ case class SortExecTransformer( } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) ( getRelNode(context, sortOrder, child.output, operatorId, readRel, validation = false), child.output) @@ -309,10 +273,6 @@ case class SortExecTransformer( throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException(s"ColumnarSortExec doesn't support doExecute") - } - override protected def withNewChildInternal(newChild: SparkPlan): SortExecTransformer = copy(child = newChild) } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala index b2b18b107360..3c2214356d7e 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/SortMergeJoinExecTransformer.scala @@ -24,7 +24,6 @@ import io.glutenproject.substrait.{JoinParams, SubstraitContext} import io.glutenproject.substrait.rel.{RelBuilder, RelNode} 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._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -34,8 +33,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.StringValue import io.substrait.proto.JoinRel -import java.util.{ArrayList => JArrayList} - import scala.collection.JavaConverters._ /** Performs a sort merge join of two child relations. */ @@ -55,13 +52,9 @@ case class SortMergeJoinExecTransformer( @transient override lazy val metrics = BackendsApiManager.getMetricsApiInstance.genSortMergeJoinTransformerMetrics(sparkContext) - val sparkConf = sparkContext.getConf - val (bufferedKeys, streamedKeys, bufferedPlan, streamedPlan) = (rightKeys, leftKeys, right, left) - override def supportsColumnar: Boolean = true - override def stringArgs: Iterator[Any] = super.stringArgs.toSeq.dropRight(1).iterator override def simpleStringWithNodeId(): String = { @@ -180,34 +173,6 @@ case class SortMergeJoinExecTransformer( getColumnarInputRDDs(streamedPlan) ++ getColumnarInputRDDs(bufferedPlan) } - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - - val curbufferedPlan: Seq[(SparkPlan, SparkPlan)] = bufferedPlan match { - case s: SortExecTransformer => - Seq((s, this)) - case c: TransformSupport if !c.isInstanceOf[SortExecTransformer] => - c.getBuildPlans - case other => - /* should be InputAdapterTransformer or others */ - Seq((other, this)) - } - streamedPlan match { - case c: TransformSupport if c.isInstanceOf[SortExecTransformer] => - curbufferedPlan ++ Seq((c, this)) - case c: TransformSupport if !c.isInstanceOf[SortExecTransformer] => - c.getBuildPlans ++ curbufferedPlan - case _ => - curbufferedPlan - } - } - - override def getStreamedLeafPlan: SparkPlan = streamedPlan match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genSortMergeJoinTransformerMetricsUpdater(metrics) @@ -291,7 +256,7 @@ case class SortMergeJoinExecTransformer( (transformContext.root, transformContext.outputAttributes, false) case _ => val readRel = RelBuilder.makeReadRel( - new JArrayList[Attribute](plan.output.asJava), + plan.output.asJava, context, -1 ) /* A special handling in Join to delay the rel registration. */ @@ -351,10 +316,6 @@ case class SortMergeJoinExecTransformer( throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException(s"ColumnarSortMergeJoinExec doesn't support doExecute") - } - override protected def withNewChildrenInternal( newLeft: SparkPlan, newRight: SparkPlan): SortMergeJoinExecTransformer = diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/TakeOrderedAndProjectExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/TakeOrderedAndProjectExecTransformer.scala index f4dbc8815950..00a5344ccc63 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/TakeOrderedAndProjectExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/TakeOrderedAndProjectExecTransformer.scala @@ -17,14 +17,13 @@ package io.glutenproject.execution import io.glutenproject.extension.GlutenPlan -import io.glutenproject.utils.ColumnarShuffleUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, NamedExpression, SortOrder} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} import org.apache.spark.sql.catalyst.util.truncatedString -import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, ColumnarInputAdapter, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.{ColumnarCollapseTransformStages, ColumnarInputAdapter, ColumnarShuffleExchangeExec, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.vectorized.ColumnarBatch @@ -97,10 +96,8 @@ case class TakeOrderedAndProjectExecTransformer( val limitStagePlan = WholeStageTransformer(limitBeforeShuffle)(transformStageCounter.incrementAndGet()) val shuffleExec = ShuffleExchangeExec(SinglePartition, limitStagePlan) - val transformedShuffleExec = ColumnarShuffleUtil.genColumnarShuffleExchange( - shuffleExec, - limitStagePlan, - shuffleExec.child.output) + val transformedShuffleExec = + ColumnarShuffleExchangeExec(shuffleExec, limitStagePlan, shuffleExec.child.output) val localSortPlan = SortExecTransformer(sortOrder, false, new ColumnarInputAdapter(transformedShuffleExec)) LimitTransformer(localSortPlan, 0, limit) diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageTransformer.scala index d62c7120d7f4..9dd6b2e0ed1e 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageTransformer.scala @@ -25,10 +25,10 @@ import io.glutenproject.metrics.{GlutenTimeMetric, MetricsUpdater, NoopMetricsUp import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} import io.glutenproject.substrait.SubstraitContext import io.glutenproject.substrait.plan.{PlanBuilder, PlanNode} -import io.glutenproject.substrait.rel.RelNode +import io.glutenproject.substrait.rel.{RelNode, SplitInfo} import io.glutenproject.utils.SubstraitPlanPrinterUtil -import org.apache.spark.SparkConf +import org.apache.spark.{Dependency, OneToOneDependency, Partition, SparkConf, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} @@ -39,6 +39,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.common.collect.Lists +import scala.collection.JavaConverters._ import scala.collection.mutable case class TransformContext( @@ -50,6 +51,13 @@ case class WholeStageTransformContext(root: PlanNode, substraitContext: Substrai trait TransformSupport extends GlutenPlan { + 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. * @@ -58,10 +66,6 @@ trait TransformSupport extends GlutenPlan { */ def columnarInputRDDs: Seq[RDD[ColumnarBatch]] - def getBuildPlans: Seq[(SparkPlan, SparkPlan)] - - def getStreamedLeafPlan: SparkPlan - def doTransform(context: SubstraitContext): TransformContext = { throw new UnsupportedOperationException( s"This operator doesn't support doTransform with SubstraitContext.") @@ -69,7 +73,7 @@ trait TransformSupport extends GlutenPlan { def metricsUpdater(): MetricsUpdater - def getColumnarInputRDDs(plan: SparkPlan): Seq[RDD[ColumnarBatch]] = { + protected def getColumnarInputRDDs(plan: SparkPlan): Seq[RDD[ColumnarBatch]] = { plan match { case c: TransformSupport => c.columnarInputRDDs @@ -79,10 +83,20 @@ trait TransformSupport extends GlutenPlan { } } +trait LeafTransformSupport extends TransformSupport with LeafExecNode { + final override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = Seq.empty +} + +trait UnaryTransformSupport extends TransformSupport with UnaryExecNode { + final override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { + getColumnarInputRDDs(child) + } +} + case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = false)( val transformStageId: Int -) extends UnaryExecNode - with TransformSupport { +) extends UnaryTransformSupport { + assert(child.isInstanceOf[TransformSupport]) // For WholeStageCodegen-like operator, only pipeline time will be handled in graph plotting. // See SparkPlanGraph.scala:205 for reference. @@ -94,13 +108,28 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f val numaBindingInfo: GlutenNumaBindingInfo = GlutenConfig.getConf.numaBindingInfo val substraitPlanLogLevel: String = GlutenConfig.getConf.substraitPlanLogLevel - private var planJson: String = "" + @transient + private var wholeStageTransformerContext: Option[WholeStageTransformContext] = None - def getPlanJson: String = { - if (log.isDebugEnabled() && planJson.isEmpty) { - logWarning("Plan in JSON string is empty. This may due to the plan has not been executed.") + def substraitPlan: PlanNode = { + if (wholeStageTransformerContext.isDefined) { + // TODO: remove this work around after we make `RelNode#toProtobuf` idempotent + // see `SubstraitContext#initSplitInfosIndex`. + wholeStageTransformerContext.get.substraitContext.initSplitInfosIndex(0) + wholeStageTransformerContext.get.root + } else { + generateWholeStageTransformContext().root } - planJson + } + + def substraitPlanJson: String = { + SubstraitPlanPrinterUtil.substraitPlanToJson(substraitPlan.toProtobuf) + } + + def nativePlanString(details: Boolean = true): String = { + BackendsApiManager.getTransformerApiInstance.getNativePlanString( + substraitPlan.toProtobuf.toByteArray, + details) } override def output: Seq[Attribute] = child.output @@ -109,8 +138,6 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def supportsColumnar: Boolean = GlutenConfig.getConf.enableColumnarIterator - override def otherCopyArgs: Seq[AnyRef] = Seq(transformStageId.asInstanceOf[Integer]) override def generateTreeString( @@ -134,9 +161,9 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f maxFields, printNodeId = printNodeId, indent) - if (verbose && planJson.nonEmpty) { + if (verbose && wholeStageTransformerContext.isDefined) { append(prefix + "Substrait plan:\n") - append(planJson) + append(substraitPlanJson) append("\n") } } @@ -146,18 +173,16 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f // See buildSparkPlanGraphNode in SparkPlanGraph.scala of Spark. override def nodeName: String = s"WholeStageCodegenTransformer ($transformStageId)" - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - child.asInstanceOf[TransformSupport].getBuildPlans - } - - override def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException("Row based execution is not supported") + override def verboseStringWithOperatorId(): String = { + val nativePlan = if (conf.getConf(GlutenConfig.INJECT_NATIVE_PLAN_STRING_TO_EXPLAIN)) { + s"Native Plan:\n${nativePlanString()}" + } else { + "" + } + super.verboseStringWithOperatorId() ++ nativePlan } - def doWholeStageTransform(): WholeStageTransformContext = { - // invoke SparkPlan.prepare to do subquery preparation etc. - super.prepare() - + private def generateWholeStageTransformContext(): WholeStageTransformContext = { val substraitContext = new SubstraitContext val childCtx = child .asInstanceOf[TransformSupport] @@ -188,13 +213,19 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f PlanBuilder.makePlan(substraitContext, Lists.newArrayList(childCtx.root), outNames) } - if (log.isDebugEnabled()) { - planJson = SubstraitPlanPrinterUtil.substraitPlanToJson(planNode.toProtobuf) - } - WholeStageTransformContext(planNode, substraitContext) } + def doWholeStageTransform(): WholeStageTransformContext = { + // invoke SparkPlan.prepare to do subquery preparation etc. + super.prepare() + val context = generateWholeStageTransformContext() + if (conf.getConf(GlutenConfig.CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT)) { + wholeStageTransformerContext = Some(context) + } + context + } + /** Find all BasicScanExecTransformers in one WholeStageTransformer */ private def findAllScanTransformers(): Seq[BasicScanExecTransformer] = { val basicScanExecTransformers = new mutable.ListBuffer[BasicScanExecTransformer]() @@ -227,10 +258,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f override def doExecuteColumnar(): RDD[ColumnarBatch] = { val pipelineTime: SQLMetric = longMetric("pipelineTime") - - val buildRelationBatchHolder: mutable.ListBuffer[ColumnarBatch] = mutable.ListBuffer() - - val inputRDDs = columnarInputRDDs + val inputRDDs = new ColumnarInputRDDsWrapper(columnarInputRDDs) // Check if BatchScan exists. val basicScanExecTransformers = findAllScanTransformers() @@ -242,35 +270,32 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f * rather than genFinalStageIterator will be invoked */ - // If these are two scan transformers, they must have same partitions, - // otherwise, exchange will be inserted. - val allScanPartitions = basicScanExecTransformers.map(_.getPartitions) - val allScanPartitionSchemas = basicScanExecTransformers.map(_.getPartitionSchemas) - val partitionLength = allScanPartitions.head.size - if (allScanPartitions.exists(_.size != partitionLength)) { - throw new GlutenException( - "The partition length of all the scan transformer are not the same.") - } + val allScanSplitInfos = getSplitInfosFromScanTransformer(basicScanExecTransformers) val (wsCxt, substraitPlanPartitions) = GlutenTimeMetric.withMillisTime { val wsCxt = doWholeStageTransform() - // the file format for each scan exec - val fileFormats = basicScanExecTransformers.map(ConverterUtils.getFileFormat) - // generate each partition of all scan exec - val substraitPlanPartitions = (0 until partitionLength).map( - i => { - val currentPartitions = allScanPartitions.map(_(i)) - BackendsApiManager.getIteratorApiInstance - .genFilePartition(i, currentPartitions, allScanPartitionSchemas, fileFormats, wsCxt) - }) + val substraitPlanPartitions = allScanSplitInfos.zipWithIndex.map { + case (splitInfos, index) => + wsCxt.substraitContext.initSplitInfosIndex(0) + wsCxt.substraitContext.setSplitInfos(splitInfos) + val substraitPlan = wsCxt.root.toProtobuf + GlutenPartition( + index, + substraitPlan.toByteArray, + splitInfos.flatMap(_.preferredLocations().asScala).toArray) + } (wsCxt, substraitPlanPartitions) - }(t => logOnLevel(substraitPlanLogLevel, s"Generating the Substrait plan took: $t ms.")) + }( + t => + logOnLevel( + substraitPlanLogLevel, + s"$nodeName generating the substrait plan took: $t ms.")) new GlutenWholeStageColumnarRDD( sparkContext, substraitPlanPartitions, - genFirstNewRDDsForBroadcast(inputRDDs, partitionLength), + inputRDDs, pipelineTime, leafMetricsUpdater().updateInputMetrics, BackendsApiManager.getMetricsApiInstance.metricsUpdatingFunction( @@ -281,6 +306,7 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f ) ) } else { + val buildRelationBatchHolder: mutable.ListBuffer[ColumnarBatch] = mutable.ListBuffer() /** * the whole stage contains NO BasicScanExecTransformer. this the default case for: @@ -290,11 +316,12 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f * result, genFinalStageIterator rather than genFirstStageIterator will be invoked */ val resCtx = GlutenTimeMetric.withMillisTime(doWholeStageTransform()) { - t => logOnLevel(substraitPlanLogLevel, s"Generating the Substrait plan took: $t ms.") + t => + logOnLevel(substraitPlanLogLevel, s"$nodeName generating the substrait plan took: $t ms.") } new WholeStageZippedPartitionsRDD( sparkContext, - genFinalNewRDDsForBroadcast(inputRDDs), + inputRDDs, numaBindingInfo, sparkConf, resCtx, @@ -311,68 +338,96 @@ case class WholeStageTransformer(child: SparkPlan, materializeInput: Boolean = f } } - override def getStreamedLeafPlan: SparkPlan = { - child.asInstanceOf[TransformSupport].getStreamedLeafPlan - } - override def metricsUpdater(): MetricsUpdater = { child match { case transformer: TransformSupport => transformer.metricsUpdater() - case _ => new NoopMetricsUpdater + case _ => NoopMetricsUpdater } } - def leafMetricsUpdater(): MetricsUpdater = { - getStreamedLeafPlan match { - case transformer: TransformSupport => transformer.metricsUpdater() - case _ => new NoopMetricsUpdater - } + private def leafMetricsUpdater(): MetricsUpdater = { + child + .find { + case t: TransformSupport if t.children.forall(!_.isInstanceOf[TransformSupport]) => true + case _ => false + } + .map(_.asInstanceOf[TransformSupport].metricsUpdater()) + .getOrElse(NoopMetricsUpdater) } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - throw new IllegalStateException( - "WholeStageTransformerExec's child should be a TransformSupport ") - } + override protected def withNewChildInternal(newChild: SparkPlan): WholeStageTransformer = + copy(child = newChild, materializeInput = materializeInput)(transformStageId) - // Recreate the broadcast build side rdd with matched partition number. - // Used when whole stage transformer contains scan. - def genFirstNewRDDsForBroadcast( - rddSeq: Seq[RDD[ColumnarBatch]], - partitions: Int): Seq[RDD[ColumnarBatch]] = { - rddSeq.map { - case rdd: BroadcastBuildSideRDD => - rdd.copy(numPartitions = partitions) - case inputRDD => - inputRDD + private def getSplitInfosFromScanTransformer( + basicScanExecTransformers: Seq[BasicScanExecTransformer]): 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 + // the substraitContext. We use transpose to do that, You can refer to + // the diagram below. + // scan1 p11 p12 p13 p14 ... p1n + // scan2 p21 p22 p23 p24 ... p2n + // transpose => + // scan1 | scan2 + // p11 | p21 => substraitContext.setSplitInfo([p11, p21]) + // p12 | p22 => substraitContext.setSplitInfo([p11, p22]) + // p13 | p23 ... + // p14 | p24 + // ... + // p1n | p2n => substraitContext.setSplitInfo([p1n, p2n]) + val allScanSplitInfos = basicScanExecTransformers.map(_.getSplitInfos) + val partitionLength = allScanSplitInfos.head.size + if (allScanSplitInfos.exists(_.size != partitionLength)) { + throw new GlutenException( + "The partition length of all the scan transformer are not the same.") } + allScanSplitInfos.transpose } +} - // Recreate the broadcast build side rdd with matched partition number. - // Used when whole stage transformer does not contain scan. - def genFinalNewRDDsForBroadcast(rddSeq: Seq[RDD[ColumnarBatch]]): Seq[RDD[ColumnarBatch]] = { - // Get the number of partitions from a non-broadcast RDD. - val nonBroadcastRDD = rddSeq.find(rdd => !rdd.isInstanceOf[BroadcastBuildSideRDD]) - if (nonBroadcastRDD.isEmpty) { - throw new GlutenException("At least one RDD should not being BroadcastBuildSideRDD") +/** + * This `columnarInputRDDs` would contain [[BroadcastBuildSideRDD]], but the dependency and + * partition of [[BroadcastBuildSideRDD]] is meaningless. [[BroadcastBuildSideRDD]] should only be + * used to hold the broadcast value and generate iterator for join. + */ +class ColumnarInputRDDsWrapper(columnarInputRDDs: Seq[RDD[ColumnarBatch]]) extends Serializable { + def getDependencies: Seq[Dependency[ColumnarBatch]] = { + assert( + columnarInputRDDs + .filterNot(_.isInstanceOf[BroadcastBuildSideRDD]) + .map(_.partitions.length) + .toSet + .size <= 1) + + columnarInputRDDs.flatMap { + case _: BroadcastBuildSideRDD => Nil + case rdd => new OneToOneDependency[ColumnarBatch](rdd) :: Nil } - rddSeq.map { - case broadcastRDD: BroadcastBuildSideRDD => - try { - broadcastRDD.getNumPartitions - broadcastRDD - } catch { - case _: Throwable => - // Recreate the broadcast build side rdd with matched partition number. - broadcastRDD.copy(numPartitions = nonBroadcastRDD.orNull.getNumPartitions) - } + } + + def getPartitions(index: Int): Seq[Partition] = { + columnarInputRDDs.filterNot(_.isInstanceOf[BroadcastBuildSideRDD]).map(_.partitions(index)) + } + + def getPartitionLength: Int = { + assert(columnarInputRDDs.nonEmpty) + val nonBroadcastRDD = columnarInputRDDs.find(!_.isInstanceOf[BroadcastBuildSideRDD]) + assert(nonBroadcastRDD.isDefined) + nonBroadcastRDD.get.partitions.length + } + + def getIterators( + inputColumnarRDDPartitions: Seq[Partition], + context: TaskContext): Seq[Iterator[ColumnarBatch]] = { + var index = 0 + columnarInputRDDs.map { + case broadcast: BroadcastBuildSideRDD => + BackendsApiManager.getIteratorApiInstance + .genBroadcastBuildSideIterator(broadcast.broadcasted, broadcast.broadCastContext) case rdd => - rdd + val it = rdd.iterator(inputColumnarRDDPartitions(index), context) + index += 1 + it } } - - override protected def withNewChildInternal(newChild: SparkPlan): WholeStageTransformer = - copy(child = newChild, materializeInput = materializeInput)(transformStageId) } diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageZippedPartitionsRDD.scala b/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageZippedPartitionsRDD.scala index a7dfb5a6ab16..1ab19aca68fa 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageZippedPartitionsRDD.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/WholeStageZippedPartitionsRDD.scala @@ -20,7 +20,7 @@ import io.glutenproject.GlutenNumaBindingInfo import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.metrics.IMetrics -import org.apache.spark.{OneToOneDependency, Partition, SparkConf, SparkContext, TaskContext} +import org.apache.spark.{Partition, SparkConf, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.vectorized.ColumnarBatch @@ -28,19 +28,13 @@ import org.apache.spark.sql.vectorized.ColumnarBatch import scala.collection.mutable private[glutenproject] class ZippedPartitionsPartition( - idx: Int, - @transient private val rdds: Seq[RDD[_]]) - extends Partition { - - override val index: Int = idx - var partitionValues: Seq[Partition] = rdds.map(rdd => rdd.partitions(idx)) - - def partitions: Seq[Partition] = partitionValues -} + override val index: Int, + val inputColumnarRDDPartitions: Seq[Partition]) + extends Partition {} class WholeStageZippedPartitionsRDD( @transient private val sc: SparkContext, - var rdds: Seq[RDD[ColumnarBatch]], + var rdds: ColumnarInputRDDsWrapper, numaBindingInfo: GlutenNumaBindingInfo, sparkConf: SparkConf, resCtx: WholeStageTransformContext, @@ -48,9 +42,11 @@ class WholeStageZippedPartitionsRDD( buildRelationBatchHolder: mutable.ListBuffer[ColumnarBatch], updateNativeMetrics: IMetrics => Unit, materializeInput: Boolean) - extends RDD[ColumnarBatch](sc, rdds.map(x => new OneToOneDependency(x))) { + extends RDD[ColumnarBatch](sc, rdds.getDependencies) { - private val genFinalStageIterator = (inputIterators: Seq[Iterator[ColumnarBatch]]) => { + override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { + val partitions = split.asInstanceOf[ZippedPartitionsPartition].inputColumnarRDDPartitions + val inputIterators: Seq[Iterator[ColumnarBatch]] = rdds.getIterators(partitions, context) BackendsApiManager.getIteratorApiInstance .genFinalStageIterator( inputIterators, @@ -64,20 +60,10 @@ class WholeStageZippedPartitionsRDD( ) } - override def compute(split: Partition, context: TaskContext): Iterator[ColumnarBatch] = { - val partitions = split.asInstanceOf[ZippedPartitionsPartition].partitions - val inputIterators: Seq[Iterator[ColumnarBatch]] = - rdds.zip(partitions).map { case (rdd, partition) => rdd.iterator(partition, context) } - genFinalStageIterator(inputIterators) - } - override def getPartitions: Array[Partition] = { - val numParts = rdds.head.partitions.length - if (!rdds.forall(rdd => rdd.partitions.length == numParts)) { - throw new IllegalArgumentException( - s"Can't zip RDDs with unequal numbers of partitions: ${rdds.map(_.partitions.length)}") + Array.tabulate[Partition](rdds.getPartitionLength) { + i => new ZippedPartitionsPartition(i, rdds.getPartitions(i)) } - Array.tabulate[Partition](numParts)(i => new ZippedPartitionsPartition(i, rdds)) } override def clearDependencies(): Unit = { diff --git a/gluten-core/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala index 4b41c42947e6..a2e4160bfaf3 100644 --- a/gluten-core/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/execution/WindowExecTransformer.scala @@ -16,28 +16,30 @@ */ package io.glutenproject.execution +import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression._ import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater -import io.glutenproject.substrait.`type`.{TypeBuilder, TypeNode} +import io.glutenproject.substrait.`type`.TypeBuilder import io.glutenproject.substrait.SubstraitContext -import io.glutenproject.substrait.expression.{ExpressionNode, WindowFunctionNode} +import io.glutenproject.substrait.expression.WindowFunctionNode import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel.{RelBuilder, RelNode} 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.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.window.WindowExecBase import org.apache.spark.sql.vectorized.ColumnarBatch -import com.google.protobuf.Any +import com.google.protobuf.{Any, StringValue} import io.substrait.proto.SortField -import java.util +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConverters._ case class WindowExecTransformer( windowExpression: Seq[NamedExpression], @@ -45,7 +47,7 @@ case class WindowExecTransformer( orderSpec: Seq[SortOrder], child: SparkPlan) extends WindowExecBase - with TransformSupport { + with UnaryTransformSupport { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics = @@ -54,8 +56,6 @@ case class WindowExecTransformer( override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genWindowTransformerMetricsUpdater(metrics) - override def supportsColumnar: Boolean = true - override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) override def requiredChildDistribution: Seq[Distribution] = { @@ -69,8 +69,11 @@ case class WindowExecTransformer( } override def requiredChildOrdering: Seq[Seq[SortOrder]] = { - if (BackendsApiManager.getSettings.requiredChildOrderingForWindow()) { - // We still need to do sort for columnar window, see `FLAGS_SkipRowSortInWindowOp` + if ( + BackendsApiManager.getSettings.requiredChildOrderingForWindow() + && GlutenConfig.getConf.veloxColumnarWindowType.equals("streaming") + ) { + // Velox StreamingWindow need to require child order. Seq(partitionSpec.map(SortOrder(_, Ascending)) ++ orderSpec) } else { Seq(Nil) @@ -81,22 +84,24 @@ case class WindowExecTransformer( override def outputPartitioning: Partitioning = child.outputPartitioning - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - throw new UnsupportedOperationException(s"This operator doesn't support getBuildPlans.") - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this + def genWindowParametersBuilder(): com.google.protobuf.Any.Builder = { + // Start with "WindowParameters:" + val windowParametersStr = new StringBuffer("WindowParameters:") + // isStreaming: 1 for streaming, 0 for sort + val isStreaming: Int = + if (GlutenConfig.getConf.veloxColumnarWindowType.equals("streaming")) 1 else 0 + + windowParametersStr + .append("isStreaming=") + .append(isStreaming) + .append("\n") + val message = StringValue + .newBuilder() + .setValue(windowParametersStr.toString) + .build() + com.google.protobuf.Any.newBuilder + .setValue(message.toByteString) + .setTypeUrl("/google.protobuf.StringValue") } def getRelNode( @@ -110,7 +115,7 @@ case class WindowExecTransformer( validation: Boolean): RelNode = { val args = context.registeredFunction // WindowFunction Expressions - val windowExpressions = new util.ArrayList[WindowFunctionNode]() + val windowExpressions = new JArrayList[WindowFunctionNode]() BackendsApiManager.getSparkPlanExecApiInstance.genWindowFunctionsNode( windowExpression, windowExpressions, @@ -119,53 +124,53 @@ case class WindowExecTransformer( ) // Partition By Expressions - val partitionsExpressions = new util.ArrayList[ExpressionNode]() - partitionSpec.map { - partitionExpr => - val exprNode = ExpressionConverter - .replaceWithExpressionTransformer(partitionExpr, attributeSeq = child.output) - .doTransform(args) - partitionsExpressions.add(exprNode) - } + val partitionsExpressions = partitionSpec + .map( + ExpressionConverter + .replaceWithExpressionTransformer(_, attributeSeq = child.output) + .doTransform(args)) + .asJava // Sort By Expressions - val sortFieldList = new util.ArrayList[SortField]() - sortOrder.map( - order => { - val builder = SortField.newBuilder() - val exprNode = ExpressionConverter - .replaceWithExpressionTransformer(order.child, attributeSeq = child.output) - .doTransform(args) - builder.setExpr(exprNode.toProtobuf) - - (order.direction.sql, order.nullOrdering.sql) match { - case ("ASC", "NULLS FIRST") => - builder.setDirectionValue(1); - case ("ASC", "NULLS LAST") => - builder.setDirectionValue(2); - case ("DESC", "NULLS FIRST") => - builder.setDirectionValue(3); - case ("DESC", "NULLS LAST") => - builder.setDirectionValue(4); - case _ => - builder.setDirectionValue(0); - } - sortFieldList.add(builder.build()) - }) + val sortFieldList = + sortOrder.map { + order => + val builder = SortField.newBuilder() + val exprNode = ExpressionConverter + .replaceWithExpressionTransformer(order.child, attributeSeq = child.output) + .doTransform(args) + builder.setExpr(exprNode.toProtobuf) + + (order.direction.sql, order.nullOrdering.sql) match { + case ("ASC", "NULLS FIRST") => + builder.setDirectionValue(1); + case ("ASC", "NULLS LAST") => + builder.setDirectionValue(2); + case ("DESC", "NULLS FIRST") => + builder.setDirectionValue(3); + case ("DESC", "NULLS LAST") => + builder.setDirectionValue(4); + case _ => + builder.setDirectionValue(0); + } + builder.build() + }.asJava if (!validation) { + val extensionNode = + ExtensionBuilder.makeAdvancedExtension(genWindowParametersBuilder.build(), null) RelBuilder.makeWindowRel( input, windowExpressions, partitionsExpressions, sortFieldList, + extensionNode, context, operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() - for (attr <- originalInputAttributes) { - inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) - } + val inputTypeNodeList = originalInputAttributes + .map(attr => ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) + .asJava val extensionNode = ExtensionBuilder.makeAdvancedExtension( Any.pack(TypeBuilder.makeStruct(false, inputTypeNodeList).toProtobuf)) @@ -231,11 +236,7 @@ case class WindowExecTransformer( } else { // This means the input is just an iterator, so an ReadRel will be created as child. // Prepare the input schema. - val attrList = new util.ArrayList[Attribute]() - for (attr <- child.output) { - attrList.add(attr) - } - val readRel = RelBuilder.makeReadRel(attrList, context, operatorId) + val readRel = RelBuilder.makeReadRel(child.output.asJava, context, operatorId) ( getRelNode( context, @@ -256,10 +257,6 @@ case class WindowExecTransformer( throw new UnsupportedOperationException(s"This operator doesn't support doExecuteColumnar().") } - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException() - } - override protected def withNewChildInternal(newChild: SparkPlan): WindowExecTransformer = copy(child = newChild) } diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/AggregateFunctionsBuilder.scala b/gluten-core/src/main/scala/io/glutenproject/expression/AggregateFunctionsBuilder.scala index cce8e4c498ea..ab6c13832ae6 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/AggregateFunctionsBuilder.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/AggregateFunctionsBuilder.scala @@ -27,19 +27,30 @@ object AggregateFunctionsBuilder { def create(args: java.lang.Object, aggregateFunc: AggregateFunction): Long = { val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] - var substraitAggFuncName = getSubstraitFunctionName(aggregateFunc) + // First handle the custom aggregate functions + val (substraitAggFuncName, inputTypes) = + if ( + ExpressionMappings.expressionExtensionTransformer.extensionExpressionsMapping.contains( + aggregateFunc.getClass) + ) { + ExpressionMappings.expressionExtensionTransformer.buildCustomAggregateFunction( + aggregateFunc) + } else { + val substraitAggFuncName = getSubstraitFunctionName(aggregateFunc) - // Check whether each backend supports this aggregate function. - if ( - !BackendsApiManager.getValidatorApiInstance.doExprValidate( - substraitAggFuncName.get, - aggregateFunc) - ) { - throw new UnsupportedOperationException( - s"Aggregate function not supported for $aggregateFunc.") - } + // Check whether each backend supports this aggregate function. + if ( + !BackendsApiManager.getValidatorApiInstance.doExprValidate( + substraitAggFuncName.get, + aggregateFunc) + ) { + throw new UnsupportedOperationException( + s"Aggregate function not supported for $aggregateFunc.") + } - val inputTypes: Seq[DataType] = aggregateFunc.children.map(child => child.dataType) + val inputTypes: Seq[DataType] = aggregateFunc.children.map(child => child.dataType) + (substraitAggFuncName, inputTypes) + } ExpressionBuilder.newScalarFunction( functionMap, diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/ArrayExpressionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/ArrayExpressionTransformer.scala index faa5a1a01ad3..a5ddad257446 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/ArrayExpressionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/ArrayExpressionTransformer.scala @@ -25,6 +25,8 @@ import org.apache.spark.sql.types._ import com.google.common.collect.Lists +import scala.collection.JavaConverters._ + case class CreateArrayTransformer( substraitExprName: String, children: Seq[ExpressionTransformer], @@ -40,12 +42,7 @@ case class CreateArrayTransformer( throw new UnsupportedOperationException(s"$original not supported yet.") } - val childNodes = new java.util.ArrayList[ExpressionNode]() - children.foreach( - child => { - val childNode = child.doTransform(args) - childNodes.add(childNode) - }) + val childNodes = children.map(_.doTransform(args)).asJava val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] val functionName = ConverterUtils.makeFuncName( @@ -87,11 +84,11 @@ case class GetArrayItemTransformer( ConverterUtils.getTypeNode(original.right.dataType, original.right.nullable)) BackendsApiManager.getSparkPlanExecApiInstance.genGetArrayItemExpressionNode( - substraitExprName: String, - functionMap: java.util.HashMap[String, java.lang.Long], - leftNode: ExpressionNode, - rightNode: ExpressionNode, - original: GetArrayItem + substraitExprName, + functionMap, + leftNode, + rightNode, + original ) } } diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/ConverterUtils.scala b/gluten-core/src/main/scala/io/glutenproject/expression/ConverterUtils.scala index 4555c88f61e5..2e1b415e3f01 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/ConverterUtils.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/ConverterUtils.scala @@ -104,32 +104,30 @@ object ConverterUtils extends Logging { getShortAttributeName(attr) + "#" + attr.exprId.id } - def collectAttributeTypeNodes(attributes: JList[Attribute]): JArrayList[TypeNode] = { + def collectAttributeTypeNodes(attributes: JList[Attribute]): JList[TypeNode] = { collectAttributeTypeNodes(attributes.asScala) } - def collectAttributeTypeNodes(attributes: Seq[Attribute]): JArrayList[TypeNode] = { - val typeList = new JArrayList[TypeNode]() - attributes.foreach(attr => typeList.add(getTypeNode(attr.dataType, attr.nullable))) - typeList + def collectAttributeTypeNodes(attributes: Seq[Attribute]): JList[TypeNode] = { + attributes.map(attr => getTypeNode(attr.dataType, attr.nullable)).asJava } - def collectAttributeNamesWithExprId(attributes: JList[Attribute]): JArrayList[String] = { + def collectAttributeNamesWithExprId(attributes: JList[Attribute]): JList[String] = { collectAttributeNamesWithExprId(attributes.asScala) } - def collectAttributeNamesWithExprId(attributes: Seq[Attribute]): JArrayList[String] = { + def collectAttributeNamesWithExprId(attributes: Seq[Attribute]): JList[String] = { collectAttributeNamesDFS(attributes)(genColumnNameWithExprId) } // TODO: This is used only by `BasicScanExecTransformer`, // perhaps we can remove this in the future and use `withExprId` version consistently. - def collectAttributeNamesWithoutExprId(attributes: Seq[Attribute]): JArrayList[String] = { + def collectAttributeNamesWithoutExprId(attributes: Seq[Attribute]): JList[String] = { collectAttributeNamesDFS(attributes)(genColumnNameWithoutExprId) } private def collectAttributeNamesDFS(attributes: Seq[Attribute])( - f: Attribute => String): JArrayList[String] = { + f: Attribute => String): JList[String] = { val nameList = new JArrayList[String]() attributes.foreach( attr => { @@ -146,7 +144,7 @@ object ConverterUtils extends Logging { nameList } - def collectStructFieldNames(dataType: DataType): JArrayList[String] = { + def collectStructFieldNames(dataType: DataType): JList[String] = { val nameList = new JArrayList[String]() dataType match { case structType: StructType => @@ -196,10 +194,10 @@ object ConverterUtils extends Logging { (DecimalType(precision, scale), isNullable(decimal.getNullability)) case Type.KindCase.STRUCT => val struct_ = substraitType.getStruct - val fields = new JArrayList[StructField] - for (typ <- struct_.getTypesList.asScala) { - val (field, nullable) = parseFromSubstraitType(typ) - fields.add(StructField("", field, nullable)) + val fields = struct_.getTypesList.asScala.map { + typ => + val (field, nullable) = parseFromSubstraitType(typ) + StructField("", field, nullable) } (StructType(fields), isNullable(substraitType.getStruct.getNullability)) case Type.KindCase.LIST => diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala index 38c86d46a6a7..7753690ddc69 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/DateTimeExpressionsTransformer.scala @@ -25,6 +25,9 @@ import org.apache.spark.sql.types._ import com.google.common.collect.Lists +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, HashMap => JHashMap} + import scala.collection.JavaConverters._ /** The extract trait for 'GetDateField' from Date */ @@ -37,7 +40,7 @@ case class ExtractDateTransformer( override def doTransform(args: java.lang.Object): ExpressionNode = { val childNode = child.doTransform(args) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionName = ConverterUtils.makeFuncName( substraitExprName, original.children.map(_.dataType), @@ -67,7 +70,7 @@ case class DateDiffTransformer( val endDateNode = endDate.doTransform(args) val startDateNode = startDate.doTransform(args) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionName = ConverterUtils.makeFuncName( substraitExprName, Seq(StringType, original.startDate.dataType, original.endDate.dataType), @@ -96,20 +99,20 @@ case class FromUnixTimeTransformer( val secNode = sec.doTransform(args) val formatNode = format.doTransform(args) - val dataTypes = if (timeZoneId != None) { + val dataTypes = if (timeZoneId.isDefined) { Seq(original.sec.dataType, original.format.dataType, StringType) } else { Seq(original.sec.dataType, original.format.dataType) } - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionId = ExpressionBuilder.newScalarFunction( functionMap, ConverterUtils.makeFuncName(substraitExprName, dataTypes)) - val expressionNodes = new java.util.ArrayList[ExpressionNode]() + val expressionNodes = new JArrayList[ExpressionNode]() expressionNodes.add(secNode) expressionNodes.add(formatNode) - if (timeZoneId != None) { + if (timeZoneId.isDefined) { expressionNodes.add(ExpressionBuilder.makeStringLiteral(timeZoneId.get)) } @@ -133,12 +136,12 @@ case class ToUnixTimestampTransformer( override def doTransform(args: java.lang.Object): ExpressionNode = { val dataTypes = Seq(original.timeExp.dataType, StringType) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] val functionId = ExpressionBuilder.newScalarFunction( functionMap, ConverterUtils.makeFuncName(substraitExprName, dataTypes)) - val expressionNodes = new java.util.ArrayList[ExpressionNode]() + val expressionNodes = new JArrayList[ExpressionNode]() val timeExpNode = timeExp.doTransform(args) expressionNodes.add(timeExpNode) val formatNode = format.doTransform(args) @@ -160,8 +163,8 @@ case class TruncTimestampTransformer( val timestampNode = timestamp.doTransform(args) val formatNode = format.doTransform(args) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] - val dataTypes = if (timeZoneId != None) { + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] + val dataTypes = if (timeZoneId.isDefined) { Seq(original.format.dataType, original.timestamp.dataType, StringType) } else { Seq(original.format.dataType, original.timestamp.dataType) @@ -171,10 +174,10 @@ case class TruncTimestampTransformer( functionMap, ConverterUtils.makeFuncName(substraitExprName, dataTypes)) - val expressionNodes = new java.util.ArrayList[ExpressionNode]() + val expressionNodes = new JArrayList[ExpressionNode]() expressionNodes.add(formatNode) expressionNodes.add(timestampNode) - if (timeZoneId != None) { + if (timeZoneId.isDefined) { expressionNodes.add(ExpressionBuilder.makeStringLiteral(timeZoneId.get)) } @@ -197,8 +200,8 @@ case class MonthsBetweenTransformer( val data2Node = date2.doTransform(args) val roundOffNode = roundOff.doTransform(args) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] - val dataTypes = if (timeZoneId != None) { + val functionMap = args.asInstanceOf[JHashMap[String, JLong]] + val dataTypes = if (timeZoneId.isDefined) { Seq(original.date1.dataType, original.date2.dataType, original.roundOff.dataType, StringType) } else { Seq(original.date1.dataType, original.date2.dataType, original.roundOff.dataType) @@ -208,11 +211,11 @@ case class MonthsBetweenTransformer( functionMap, ConverterUtils.makeFuncName(substraitExprName, dataTypes)) - val expressionNodes = new java.util.ArrayList[ExpressionNode]() + val expressionNodes = new JArrayList[ExpressionNode]() expressionNodes.add(date1Node) expressionNodes.add(data2Node) expressionNodes.add(roundOffNode) - if (timeZoneId != None) { + if (timeZoneId.isDefined) { expressionNodes.add(ExpressionBuilder.makeStringLiteral(timeZoneId.get)) } diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionConverter.scala b/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionConverter.scala index 729526acfe18..9cd2b16f14fd 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionConverter.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionConverter.scala @@ -19,11 +19,12 @@ package io.glutenproject.expression import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution.{ColumnarToRowExecBase, WholeStageTransformer} +import io.glutenproject.extension.GlutenPlan import io.glutenproject.test.TestStats +import io.glutenproject.utils.DecimalArithmeticUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} -import org.apache.spark.sql.catalyst.analysis.DecimalPrecision import org.apache.spark.sql.catalyst.expressions.{BinaryArithmetic, _} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero @@ -44,195 +45,6 @@ trait Transformable extends Expression { } object ExpressionConverter extends SQLConfHelper with Logging { - - /** - * Remove the Cast when child is PromotePrecision and PromotePrecision is Cast(Decimal, Decimal) - * - * @param arithmeticExpr - * BinaryArithmetic left or right - * @return - * expression removed child PromotePrecision->Cast - */ - private def removeCastForDecimal(arithmeticExpr: Expression): Expression = { - arithmeticExpr match { - case precision: PromotePrecision => - precision.child match { - case cast: Cast - if cast.dataType.isInstanceOf[DecimalType] - && cast.child.dataType.isInstanceOf[DecimalType] => - cast.child - case _ => arithmeticExpr - } - case _ => arithmeticExpr - } - } - - private def isPromoteCastIntegral(expr: Expression): Boolean = { - expr match { - case precision: PromotePrecision => - precision.child match { - case cast: Cast if cast.dataType.isInstanceOf[DecimalType] => - cast.child.dataType match { - case IntegerType | ByteType | ShortType | LongType => true - case _ => false - } - case _ => false - } - case _ => false - } - } - - private def isPromoteCast(expr: Expression): Boolean = { - expr match { - case precision: PromotePrecision => - precision.child match { - case cast: Cast if cast.dataType.isInstanceOf[DecimalType] => true - case _ => false - } - case _ => false - } - } - - private def rescaleCastForOneSide(expr: Expression): Expression = { - expr match { - case precision: PromotePrecision => - precision.child match { - case castInt: Cast - if castInt.dataType.isInstanceOf[DecimalType] && - BackendsApiManager.getSettings.rescaleDecimalIntegralExpression() => - castInt.child.dataType match { - case IntegerType | ByteType | ShortType => - precision.withNewChildren(Seq(Cast(castInt.child, DecimalType(10, 0)))) - case LongType => - precision.withNewChildren(Seq(Cast(castInt.child, DecimalType(20, 0)))) - case _ => expr - } - case _ => expr - } - case _ => expr - } - } - - private def checkIsWiderType( - left: DecimalType, - right: DecimalType, - wider: DecimalType): Boolean = { - val widerType = DecimalPrecision.widerDecimalType(left, right) - widerType.equals(wider) - } - - private def rescaleCastForDecimal( - left: Expression, - right: Expression): (Expression, Expression) = { - if (!BackendsApiManager.getSettings.rescaleDecimalIntegralExpression()) { - return (left, right) - } - // decimal * cast int - if (!isPromoteCast(left)) { // have removed PromotePrecision(Cast(DecimalType)) - if (isPromoteCastIntegral(right)) { - val newRight = rescaleCastForOneSide(right) - val isWiderType = checkIsWiderType( - left.dataType.asInstanceOf[DecimalType], - newRight.dataType.asInstanceOf[DecimalType], - right.dataType.asInstanceOf[DecimalType]) - if (isWiderType) { - (left, newRight) - } else { - (left, right) - } - } else { - (left, right) - } - // cast int * decimal - } else if (!isPromoteCast(right)) { - if (isPromoteCastIntegral(left)) { - val newLeft = rescaleCastForOneSide(left) - val isWiderType = checkIsWiderType( - newLeft.dataType.asInstanceOf[DecimalType], - right.dataType.asInstanceOf[DecimalType], - left.dataType.asInstanceOf[DecimalType]) - if (isWiderType) { - (newLeft, right) - } else { - (left, right) - } - } else { - (left, right) - } - } else { - // cast int * cast int, usually user defined cast - (left, right) - } - } - - // For decimal * 10 case, dec will be Decimal(38, 18), then the result precision is wrong, - // so here we will get the real precision and scale of the literal - private def getNewPrecisionScale(dec: Decimal): (Integer, Integer) = { - val input = dec.abs.toString() - val dotIndex = input.indexOf(".") - if (dotIndex == -1) { - return (input.length, 0) - } - - if (dec.toBigDecimal.isValidLong) { - return (dotIndex, 0) - } - - (dec.precision, dec.scale) - } - - // change the precision and scale to literal actual precision and scale, otherwise the result - // precision loss - private def rescaleLiteral(arithmeticExpr: BinaryArithmetic): BinaryArithmetic = { - if ( - arithmeticExpr.left.isInstanceOf[PromotePrecision] - && arithmeticExpr.right.isInstanceOf[Literal] - ) { - val lit = arithmeticExpr.right.asInstanceOf[Literal] - lit.value match { - case decLit: Decimal => - val (precision, scale) = getNewPrecisionScale(decLit) - if (precision != decLit.precision || scale != decLit.scale) { - arithmeticExpr - .withNewChildren(Seq(arithmeticExpr.left, Cast(lit, DecimalType(precision, scale)))) - .asInstanceOf[BinaryArithmetic] - } else arithmeticExpr - case _ => arithmeticExpr - } - } else if ( - arithmeticExpr.right.isInstanceOf[PromotePrecision] - && arithmeticExpr.left.isInstanceOf[Literal] - ) { - val lit = arithmeticExpr.left.asInstanceOf[Literal] - lit.value match { - case decLit: Decimal => - val (precision, scale) = getNewPrecisionScale(decLit) - if (precision != decLit.precision || scale != decLit.scale) { - arithmeticExpr - .withNewChildren(Seq(Cast(lit, DecimalType(precision, scale)), arithmeticExpr.right)) - .asInstanceOf[BinaryArithmetic] - } else arithmeticExpr - case _ => arithmeticExpr - } - } else { - arithmeticExpr - } - } - - // If casting between DecimalType, unnecessary cast is skipped to avoid data loss, - // because argument input type of "cast" is actually the res type of "+-*/". - // Cast will use a wider input type, then calculated a less scale result type than vanilla spark - private def isDecimalArithmetic(expr: BinaryArithmetic): Boolean = { - expr match { - case b if b.children.forall(_.dataType.isInstanceOf[DecimalType]) => - b match { - case _: Divide | _: Multiply | _: Add | _: Subtract | _: Remainder | _: Pmod => true - case _ => false - } - case _ => false - } - } - def replacePythonUDFWithExpressionTransformer( udf: PythonUDF, attributeSeq: Seq[Attribute]): ExpressionTransformer = { @@ -267,7 +79,7 @@ object ExpressionConverter extends SQLConfHelper with Logging { expr: Expression, attributeSeq: Seq[Attribute]): ExpressionTransformer = { logDebug( - s"replaceWithExpressionTransformer expr: $expr class: ${expr.getClass}} " + + s"replaceWithExpressionTransformer expr: $expr class: ${expr.getClass} " + s"name: ${expr.prettyName}") expr match { @@ -288,10 +100,7 @@ object ExpressionConverter extends SQLConfHelper with Logging { } // Check whether each backend supports this expression - if ( - GlutenConfig.getConf.enableAnsiMode || - !BackendsApiManager.getValidatorApiInstance.doExprValidate(substraitExprName, expr) - ) { + if (!BackendsApiManager.getValidatorApiInstance.doExprValidate(substraitExprName, expr)) { throw new UnsupportedOperationException(s"Not supported: $expr.") } expr match { @@ -479,7 +288,7 @@ object ExpressionConverter extends SQLConfHelper with Logging { getStructField.ordinal, getStructField) case t: StringTranslate => - StringTranslateTransformer( + BackendsApiManager.getSparkPlanExecApiInstance.genStringTranslateTransformer( substraitExprName, replaceWithExpressionTransformer(t.srcExpr, attributeSeq), replaceWithExpressionTransformer(t.matchingExpr, attributeSeq), @@ -590,24 +399,40 @@ object ExpressionConverter extends SQLConfHelper with Logging { substraitExprName, replaceWithExpressionTransformer(expr.children.head, attributeSeq), expr) - case b: BinaryArithmetic if isDecimalArithmetic(b) => - if (!conf.decimalOperationsAllowPrecisionLoss) { + case b: BinaryArithmetic if DecimalArithmeticUtil.isDecimalArithmetic(b) => + // PrecisionLoss=true: velox support / ch not support + // PrecisionLoss=false: velox not support / ch support + // TODO ch support PrecisionLoss=true + if (!BackendsApiManager.getSettings.allowDecimalArithmetic) { throw new UnsupportedOperationException( - s"Not support ${SQLConf.DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key} false mode") + s"Not support ${SQLConf.DECIMAL_OPERATIONS_ALLOW_PREC_LOSS.key} " + + s"${conf.decimalOperationsAllowPrecisionLoss} mode") } val rescaleBinary = if (BackendsApiManager.getSettings.rescaleDecimalLiteral) { - rescaleLiteral(b) + DecimalArithmeticUtil.rescaleLiteral(b) } else { b } - val (left, right) = rescaleCastForDecimal( - removeCastForDecimal(rescaleBinary.left), - removeCastForDecimal(rescaleBinary.right)) - GenericExpressionTransformer( + val (left, right) = DecimalArithmeticUtil.rescaleCastForDecimal( + DecimalArithmeticUtil.removeCastForDecimal(rescaleBinary.left), + DecimalArithmeticUtil.removeCastForDecimal(rescaleBinary.right)) + val leftChild = replaceWithExpressionTransformer(left, attributeSeq) + val rightChild = replaceWithExpressionTransformer(right, attributeSeq) + + val resultType = DecimalArithmeticUtil.getResultTypeForOperation( + DecimalArithmeticUtil.getOperationType(b), + DecimalArithmeticUtil + .getResultType(leftChild) + .getOrElse(left.dataType.asInstanceOf[DecimalType]), + DecimalArithmeticUtil + .getResultType(rightChild) + .getOrElse(right.dataType.asInstanceOf[DecimalType]) + ) + DecimalArithmeticExpressionTransformer( substraitExprName, - Seq( - replaceWithExpressionTransformer(left, attributeSeq), - replaceWithExpressionTransformer(right, attributeSeq)), + leftChild, + rightChild, + resultType, b) case e: Transformable => val childrenTransformers = e.children.map(replaceWithExpressionTransformer(_, attributeSeq)) @@ -637,16 +462,16 @@ object ExpressionConverter extends SQLConfHelper with Logging { // get WholeStageTransformer directly case c2r: ColumnarToRowExecBase => c2r.child // in fallback case - case codeGen: WholeStageCodegenExec => - if (codeGen.child.isInstanceOf[ColumnarToRowExec]) { + case plan: UnaryExecNode if !plan.isInstanceOf[GlutenPlan] => + if (plan.child.isInstanceOf[ColumnarToRowExec]) { val wholeStageTransformer = exchange.find(_.isInstanceOf[WholeStageTransformer]) if (wholeStageTransformer.nonEmpty) { wholeStageTransformer.get } else { - BackendsApiManager.getSparkPlanExecApiInstance.genRowToColumnarExec(codeGen) + BackendsApiManager.getSparkPlanExecApiInstance.genRowToColumnarExec(plan) } } else { - BackendsApiManager.getSparkPlanExecApiInstance.genRowToColumnarExec(codeGen) + BackendsApiManager.getSparkPlanExecApiInstance.genRowToColumnarExec(plan) } } ColumnarBroadcastExchangeExec(exchange.mode, newChild) diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionMappings.scala b/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionMappings.scala index a388a9e02538..705ec28bf364 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionMappings.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/ExpressionMappings.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.optimizer.NormalizeNaNAndZero import org.apache.spark.sql.execution.ScalarSubquery -import org.apache.spark.sql.execution.datasources.FileFormatWriter.Empty2Null object ExpressionMappings { @@ -85,10 +84,10 @@ object ExpressionMappings { Sig[Substring](SUBSTRING), Sig[SubstringIndex](SUBSTRING_INDEX), Sig[ConcatWs](CONCAT_WS), + Sig[Left](LEFT), Sig[StringRepeat](REPEAT), Sig[StringTranslate](TRANSLATE), Sig[StringSpace](SPACE), - Sig[Empty2Null](EMPTY2NULL), Sig[InitCap](INITCAP), Sig[Overlay](OVERLAY), Sig[Conv](CONV), @@ -247,6 +246,8 @@ object ExpressionMappings { Sig[Count](COUNT), Sig[Min](MIN), Sig[Max](MAX), + Sig[MaxBy](MAX_BY), + Sig[MinBy](MIN_BY), Sig[StddevSamp](STDDEV_SAMP), Sig[StddevPop](STDDEV_POP), Sig[CollectList](COLLECT_LIST), diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/LambdaFunctionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/LambdaFunctionTransformer.scala index ce0466afc19b..05ba330256c7 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/LambdaFunctionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/LambdaFunctionTransformer.scala @@ -20,8 +20,6 @@ import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} import org.apache.spark.sql.catalyst.expressions.LambdaFunction -import java.util.ArrayList - case class LambdaFunctionTransformer( substraitExprName: String, function: ExpressionTransformer, @@ -42,7 +40,7 @@ case class LambdaFunctionTransformer( substraitExprName, Seq(original.dataType), ConverterUtils.FunctionConfig.OPT)) - val expressionNodes = new ArrayList[ExpressionNode] + val expressionNodes = new java.util.ArrayList[ExpressionNode] expressionNodes.add(function.doTransform(args)) arguments.foreach(argument => expressionNodes.add(argument.doTransform(args))) val typeNode = ConverterUtils.getTypeNode(original.dataType, original.nullable) diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/PredicateExpressionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/PredicateExpressionTransformer.scala index 4994e69c1298..9e48646a6b82 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/PredicateExpressionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/PredicateExpressionTransformer.scala @@ -23,6 +23,8 @@ import io.glutenproject.substrait.expression.{ExpressionBuilder, ExpressionNode} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ +import com.google.common.collect.Lists + import scala.collection.JavaConverters._ case class InTransformer( @@ -96,3 +98,27 @@ case class LikeTransformer( ExpressionBuilder.makeScalarFunction(functionId, expressionNodes.toList.asJava, typeNode) } } + +case class DecimalArithmeticExpressionTransformer( + substraitExprName: String, + left: ExpressionTransformer, + right: ExpressionTransformer, + resultType: DecimalType, + original: Expression) + extends ExpressionTransformer { + override def doTransform(args: java.lang.Object): ExpressionNode = { + val leftNode = left.doTransform(args) + val rightNode = right.doTransform(args) + val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] + val functionId = ExpressionBuilder.newScalarFunction( + functionMap, + ConverterUtils.makeFuncName( + substraitExprName, + original.children.map(_.dataType), + FunctionConfig.OPT)) + + val expressionNodes = Lists.newArrayList(leftNode, rightNode) + val typeNode = ConverterUtils.getTypeNode(resultType, original.nullable) + ExpressionBuilder.makeScalarFunction(functionId, expressionNodes, typeNode) + } +} diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/StringExpressionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/StringExpressionTransformer.scala index 709c9f8d8018..93d1a7bedf17 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/StringExpressionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/StringExpressionTransformer.scala @@ -48,39 +48,6 @@ case class String2TrimExpressionTransformer( } } -case class StringTranslateTransformer( - substraitExprName: String, - srcExpr: ExpressionTransformer, - matchingExpr: ExpressionTransformer, - replaceExpr: ExpressionTransformer, - original: StringTranslate) - extends ExpressionTransformer { - - override def doTransform(args: java.lang.Object): ExpressionNode = { - // In CH, translateUTF8 requires matchingExpr and replaceExpr argument have the same length - val matchingNode = matchingExpr.doTransform(args) - val replaceNode = replaceExpr.doTransform(args) - if ( - !matchingNode.isInstanceOf[StringLiteralNode] || - !replaceNode.isInstanceOf[StringLiteralNode] - ) { - throw new UnsupportedOperationException(s"$original not supported yet.") - } - - val matchingLiteral = matchingNode.asInstanceOf[StringLiteralNode].getValue - val replaceLiteral = replaceNode.asInstanceOf[StringLiteralNode].getValue - if (matchingLiteral.length() != replaceLiteral.length()) { - throw new UnsupportedOperationException(s"$original not supported yet.") - } - - GenericExpressionTransformer( - substraitExprName, - Seq(srcExpr, matchingExpr, replaceExpr), - original) - .doTransform(args) - } -} - case class RegExpReplaceTransformer( substraitExprName: String, subject: ExpressionTransformer, diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/StructExpressionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/StructExpressionTransformer.scala index 363408e567c8..e5af5400ba88 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/StructExpressionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/StructExpressionTransformer.scala @@ -31,7 +31,7 @@ case class GetStructFieldTransformer( original: GetStructField) extends ExpressionTransformer { - override def doTransform(args: Object): ExpressionNode = { + override def doTransform(args: java.lang.Object): ExpressionNode = { val childNode = childTransformer.doTransform(args) childNode match { case node: StructLiteralNode => diff --git a/gluten-core/src/main/scala/io/glutenproject/expression/UnaryExpressionTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/expression/UnaryExpressionTransformer.scala index 7c013aacdf8b..627f3a4fa854 100644 --- a/gluten-core/src/main/scala/io/glutenproject/expression/UnaryExpressionTransformer.scala +++ b/gluten-core/src/main/scala/io/glutenproject/expression/UnaryExpressionTransformer.scala @@ -16,6 +16,7 @@ */ package io.glutenproject.expression +import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.expression.ConverterUtils.FunctionConfig import io.glutenproject.substrait.`type`.ListNode import io.glutenproject.substrait.`type`.MapNode @@ -163,22 +164,13 @@ case class CheckOverflowTransformer( extends ExpressionTransformer { override def doTransform(args: java.lang.Object): ExpressionNode = { - val childNode = child.doTransform(args) - val functionMap = args.asInstanceOf[java.util.HashMap[String, java.lang.Long]] - val functionId = ExpressionBuilder.newScalarFunction( - functionMap, - ConverterUtils.makeFuncName( - substraitExprName, - Seq(original.dataType, BooleanType), - FunctionConfig.OPT)) - - // just make a fake toType value, because native engine cannot accept datatype itself - val toTypeNodes = ExpressionBuilder.makeDecimalLiteral( - new Decimal().set(0, original.dataType.precision, original.dataType.scale)) - val expressionNodes = - Lists.newArrayList(childNode, new BooleanLiteralNode(original.nullOnOverflow), toTypeNodes) - val typeNode = ConverterUtils.getTypeNode(original.dataType, original.nullable) - ExpressionBuilder.makeScalarFunction(functionId, expressionNodes, typeNode) + BackendsApiManager.getTransformerApiInstance.createCheckOverflowExprNode( + args, + substraitExprName, + child.doTransform(args), + original.dataType, + original.nullable, + original.nullOnOverflow) } } diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala b/gluten-core/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala index 2497439725b7..3c12d0c56e22 100644 --- a/gluten-core/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala +++ b/gluten-core/src/main/scala/io/glutenproject/extension/ColumnarOverrides.scala @@ -22,7 +22,8 @@ import io.glutenproject.execution._ import io.glutenproject.expression.ExpressionConverter import io.glutenproject.extension.columnar._ import io.glutenproject.metrics.GlutenTimeMetric -import io.glutenproject.utils.{ColumnarShuffleUtil, LogLevelUtil, PhysicalPlanSelector} +import io.glutenproject.sql.shims.SparkShimLoader +import io.glutenproject.utils.{LogLevelUtil, PhysicalPlanSelector} import org.apache.spark.api.python.EvalPythonExecTransformer import org.apache.spark.internal.Logging @@ -110,9 +111,7 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) // If the child is transformable, transform aggregation as well. logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") transformHashAggregate() - case q: QueryStageExec if InMemoryTableScanHelper.isGlutenTableCache(q) => - transformHashAggregate() - case i: InMemoryTableScanExec if InMemoryTableScanHelper.isGlutenTableCache(i) => + case p: SparkPlan if InMemoryTableScanHelper.isGlutenTableCache(p) => transformHashAggregate() case _ => // If the child is not transformable, transform the grandchildren only. @@ -396,15 +395,12 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) case HashPartitioning(exprs, _) => val projectChild = getProjectWithHash(exprs, child) if (projectChild.supportsColumnar) { - ColumnarShuffleUtil.genColumnarShuffleExchange( - plan, - projectChild, - projectChild.output.drop(1)) + ColumnarShuffleExchangeExec(plan, projectChild, projectChild.output.drop(1)) } else { plan.withNewChildren(Seq(child)) } case _ => - ColumnarShuffleUtil.genColumnarShuffleExchange(plan, child, null) + ColumnarShuffleExchangeExec(plan, child, null) } } else if ( BackendsApiManager.getSettings.supportShuffleWithProject( @@ -418,7 +414,7 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) if (newChild.supportsColumnar) { val newPlan = ShuffleExchangeExec(newPartitioning, newChild, plan.shuffleOrigin) // the new projections columns are appended at the end. - ColumnarShuffleUtil.genColumnarShuffleExchange( + ColumnarShuffleExchangeExec( newPlan, newChild, newChild.output.dropRight(projectColumnNumber)) @@ -427,10 +423,10 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) plan.withNewChildren(Seq(child)) } } else { - ColumnarShuffleUtil.genColumnarShuffleExchange(plan, child, null) + ColumnarShuffleExchangeExec(plan, child, null) } } else { - ColumnarShuffleUtil.genColumnarShuffleExchange(plan, child, null) + ColumnarShuffleExchangeExec(plan, child, null) } } else { plan.withNewChildren(Seq(child)) @@ -478,18 +474,6 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) left, right, isNullAwareAntiJoin = plan.isNullAwareAntiJoin) - case plan: AQEShuffleReadExec - if BackendsApiManager.getSettings.supportColumnarShuffleExec() => - plan.child match { - case ShuffleQueryStageExec(_, _: ColumnarShuffleExchangeExec, _) => - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - ColumnarAQEShuffleReadExec(plan.child, plan.partitionSpecs) - case ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: ColumnarShuffleExchangeExec), _) => - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - ColumnarAQEShuffleReadExec(plan.child, plan.partitionSpecs) - case _ => - plan - } case plan: WindowExec => WindowExecTransformer( plan.windowExpression, @@ -517,9 +501,6 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") val child = replaceWithTransformerPlan(plan.child) EvalPythonExecTransformer(plan.udfs, plan.resultAttrs, child) - case plan if HiveTableScanExecTransformer.isHiveTableScan(plan) => - logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") - BackendsApiManager.getSparkPlanExecApiInstance.genHiveTableScanExecTransformer(plan) case p => logDebug(s"Transformation for ${p.getClass} is currently not supported.") val children = plan.children.map(replaceWithTransformerPlan) @@ -580,7 +561,12 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) case _ => ExpressionConverter.transformDynamicPruningExpr(plan.runtimeFilters, reuseSubquery) } - val transformer = new BatchScanExecTransformer(plan.output, plan.scan, newPartitionFilters) + val transformer = new BatchScanExecTransformer( + plan.output, + plan.scan, + newPartitionFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(plan)) + val validationResult = transformer.doValidate() if (validationResult.isValid) { logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") @@ -600,8 +586,10 @@ case class TransformPreOverrides(isAdaptiveContext: Boolean) BackendsApiManager.getSparkPlanExecApiInstance.genHiveTableScanExecTransformer(plan) val validateResult = hiveTableScanExecTransformer.doValidate() if (validateResult.isValid) { + logDebug(s"Columnar Processing for ${plan.getClass} is currently supported.") return hiveTableScanExecTransformer } + logDebug(s"Columnar Processing for ${plan.getClass} is currently unsupported.") val newSource = HiveTableScanExecTransformer.copyWith(plan, newPartitionFilters) TransformHints.tagNotTransformable(newSource, validateResult.reason.get) newSource @@ -656,6 +644,9 @@ case class TransformPostOverrides(isAdaptiveContext: Boolean) extends Rule[Spark replaceWithTransformerPlan(child) case ColumnarToRowExec(child: BroadcastQueryStageExec) => replaceWithTransformerPlan(child) + // `InMemoryTableScanExec` internally supports ColumnarToRow + case ColumnarToRowExec(child: SparkPlan) if InMemoryTableScanHelper.isGlutenTableCache(child) => + child case plan: ColumnarToRowExec => transformColumnarToRowExec(plan) case r: SparkPlan @@ -664,7 +655,8 @@ case class TransformPostOverrides(isAdaptiveContext: Boolean) extends Rule[Spark // This is a fix for when DPP and AQE both enabled, // ColumnarExchange maybe child as a Row SparkPlan r.withNewChildren(r.children.map { - case c: ColumnarToRowExec => + // `InMemoryTableScanExec` internally supports ColumnarToRow + case c: ColumnarToRowExec if !InMemoryTableScanHelper.isGlutenTableCache(c.child) => transformColumnarToRowExec(c) case other => replaceWithTransformerPlan(other) @@ -814,6 +806,7 @@ case class ColumnarOverrideRules(session: SparkSession) (_: SparkSession) => FallbackEmptySchemaRelation(), (_: SparkSession) => AddTransformHintRule(), (_: SparkSession) => TransformPreOverrides(isAdaptiveContext), + (spark: SparkSession) => RewriteTransformer(spark), (_: SparkSession) => EnsureLocalSortRequirements ) ::: BackendsApiManager.getSparkPlanExecApiInstance.genExtendedColumnarPreRules() ::: diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/ExpandFallbackPolicy.scala b/gluten-core/src/main/scala/io/glutenproject/extension/ExpandFallbackPolicy.scala index e15564700895..98c0e9b4c77c 100644 --- a/gluten-core/src/main/scala/io/glutenproject/extension/ExpandFallbackPolicy.scala +++ b/gluten-core/src/main/scala/io/glutenproject/extension/ExpandFallbackPolicy.scala @@ -17,7 +17,6 @@ package io.glutenproject.extension import io.glutenproject.GlutenConfig -import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution.BroadcastHashJoinExecTransformer import io.glutenproject.extension.columnar.TransformHints @@ -25,11 +24,11 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{ColumnarBroadcastExchangeExec, ColumnarShuffleExchangeExec, ColumnarToRowExec, CommandResultExec, LeafExecNode, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, BroadcastQueryStageExec, ColumnarAQEShuffleReadExec, QueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.{ColumnarBroadcastExchangeExec, ColumnarToRowExec, CommandResultExec, LeafExecNode, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, BroadcastQueryStageExec, QueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.ExecutedCommandExec -import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} +import org.apache.spark.sql.execution.exchange.Exchange // spotless:off /** @@ -68,36 +67,115 @@ import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkPlan) extends Rule[SparkPlan] { - private def countFallbacks(plan: SparkPlan): Int = { + private def countFallback(plan: SparkPlan): Int = { var fallbacks = 0 - def countFallback(plan: SparkPlan): Unit = { + def countFallbackInternal(plan: SparkPlan): Unit = { plan match { case _: QueryStageExec => // Another stage. case _: CommandResultExec | _: ExecutedCommandExec => // ignore // we plan exchange to columnar exchange in columnar rules and the exchange does not // support columnar, so the output columnar is always false in AQE postStageCreationRules case ColumnarToRowExec(s: Exchange) if isAdaptiveContext => - countFallback(s) + countFallbackInternal(s) case u: UnaryExecNode if !u.isInstanceOf[GlutenPlan] && InMemoryTableScanHelper.isGlutenTableCache(u.child) => // Vanilla Spark plan will call `InMemoryTableScanExec.convertCachedBatchToInternalRow` // which is a kind of `ColumnarToRowExec`. fallbacks = fallbacks + 1 - countFallback(u.child) + countFallbackInternal(u.child) case ColumnarToRowExec(p: GlutenPlan) => logDebug(s"Find a columnar to row for gluten plan:\n$p") fallbacks = fallbacks + 1 - countFallback(p) + countFallbackInternal(p) + case leafPlan: LeafExecNode if InMemoryTableScanHelper.isGlutenTableCache(leafPlan) => case leafPlan: LeafExecNode if !leafPlan.isInstanceOf[GlutenPlan] => // Possible fallback for leaf node. fallbacks = fallbacks + 1 - case p => p.children.foreach(countFallback) + case p => p.children.foreach(countFallbackInternal) } } - countFallback(plan) + countFallbackInternal(plan) fallbacks } + /** + * 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 + * ----------- | --------------- last stage + * HashAggregateTransformer + * | + * ColumnarToRow + * | + * Project + * + * To illustrate the effect if cost is not taken into account, here is spark plan + * after applying whole stage fallback policy (threshold = 1): + * + * ColumnarExchange + * ----------- | --------------- last stage + * ColumnarToRow + * | + * HashAggregate + * | + * Project + * + * So by considering the cost, the fallback policy will not be applied. + * + * spotless:on + */ + private def countStageFallbackCost(plan: SparkPlan): Int = { + var stageFallbackCost = 0 + + /** + * 1) Find a Gluten plan whose child is InMemoryTableScanExec. Then, increase stageFallbackCost + * if InMemoryTableScanExec is gluten's table cache and decrease stageFallbackCost if not. 2) + * Find a Gluten plan whose child is QueryStageExec. Then, increase stageFallbackCost if the + * last query stage's plan is GlutenPlan and decrease stageFallbackCost if not. + */ + def countStageFallbackCostInternal(plan: SparkPlan): Unit = { + plan match { + case _: GlutenPlan if plan.children.find(_.isInstanceOf[InMemoryTableScanExec]).isDefined => + plan.children + .filter(_.isInstanceOf[InMemoryTableScanExec]) + .foreach { + // For this case, table cache will internally execute ColumnarToRow if + // we make the stage fall back. + case child if InMemoryTableScanHelper.isGlutenTableCache(child) => + stageFallbackCost = stageFallbackCost + 1 + // For other case, table cache will save internal RowToColumnar if we make + // the stage fall back. + case _ => + stageFallbackCost = stageFallbackCost - 1 + } + case _: GlutenPlan if plan.children.find(_.isInstanceOf[QueryStageExec]).isDefined => + plan.children + .filter(_.isInstanceOf[QueryStageExec]) + .foreach { + case stage: QueryStageExec + if stage.plan.isInstanceOf[GlutenPlan] || + // For TableCacheQueryStageExec since spark 3.5. + InMemoryTableScanHelper.isGlutenTableCache(stage) => + stageFallbackCost = stageFallbackCost + 1 + // For other cases, RowToColumnar will be removed if stage falls back, so reduce + // the cost. + case _ => + stageFallbackCost = stageFallbackCost - 1 + } + case _ => plan.children.foreach(countStageFallbackCostInternal) + } + } + countStageFallbackCostInternal(plan) + stageFallbackCost + } + private def hasColumnarBroadcastExchangeWithJoin(plan: SparkPlan): Boolean = { def isColumnarBroadcastExchange(p: SparkPlan): Boolean = p match { case BroadcastQueryStageExec(_, _: ColumnarBroadcastExchangeExec, _) => true @@ -118,7 +196,7 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP GlutenConfig.getConf.wholeStageFallbackThreshold } else if (plan.find(_.isInstanceOf[AdaptiveSparkPlanExec]).isDefined) { // if we are here, that means we are now at `QueryExecution.preparations` and - // AQE is actually applied. We do nothing for this case, and later in + // AQE is actually not applied. We do nothing for this case, and later in // AQE we can check `wholeStageFallbackThreshold`. return None } else { @@ -134,11 +212,15 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP return None } - val numFallback = countFallbacks(plan) - if (numFallback >= fallbackThreshold) { + val netFallbackNum = if (isAdaptiveContext) { + countFallback(plan) - countStageFallbackCost(plan) + } else { + countFallback(plan) + } + if (netFallbackNum >= fallbackThreshold) { Some( - s"Fall back the plan due to fallback number $numFallback, " + - s"threshold $fallbackThreshold") + s"Fallback policy is taking effect, net fallback number: $netFallbackNum, " + + s"threshold: $fallbackThreshold") } else { None } @@ -146,30 +228,16 @@ case class ExpandFallbackPolicy(isAdaptiveContext: Boolean, originalPlan: SparkP private def fallbackToRowBasedPlan(): SparkPlan = { val transformPostOverrides = TransformPostOverrides(isAdaptiveContext) - val planWithReplacedAQERead = originalPlan.transform { - case plan: AQEShuffleReadExec - if BackendsApiManager.getSettings.supportColumnarShuffleExec() => - plan.child match { - case ShuffleQueryStageExec(_, _: ColumnarShuffleExchangeExec, _) => - ColumnarAQEShuffleReadExec(plan.child, plan.partitionSpecs) - case ShuffleQueryStageExec(_, ReusedExchangeExec(_, _: ColumnarShuffleExchangeExec), _) => - ColumnarAQEShuffleReadExec(plan.child, plan.partitionSpecs) - case _ => - plan - } - } - val planWithColumnarToRow = InsertTransitions.insertTransitions(planWithReplacedAQERead, false) + val planWithColumnarToRow = InsertTransitions.insertTransitions(originalPlan, false) planWithColumnarToRow.transform { case c2r @ ColumnarToRowExec(_: ShuffleQueryStageExec) => transformPostOverrides.transformColumnarToRowExec(c2r) - case c2r @ ColumnarToRowExec(_: ColumnarAQEShuffleReadExec) => + case c2r @ ColumnarToRowExec(_: AQEShuffleReadExec) => transformPostOverrides.transformColumnarToRowExec(c2r) - case ColumnarToRowExec(q: QueryStageExec) if InMemoryTableScanHelper.isGlutenTableCache(q) => - q - case ColumnarToRowExec(i: InMemoryTableScanExec) - if InMemoryTableScanHelper.isGlutenTableCache(i) => - // `InMemoryTableScanExec` itself supports columnar to row - i + // `InMemoryTableScanExec` itself supports columnar to row + case ColumnarToRowExec(child: SparkPlan) + if InMemoryTableScanHelper.isGlutenTableCache(child) => + child } } diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/ExpressionExtensionTrait.scala b/gluten-core/src/main/scala/io/glutenproject/extension/ExpressionExtensionTrait.scala index 89452bac2a6f..8542a7fdbc86 100644 --- a/gluten-core/src/main/scala/io/glutenproject/extension/ExpressionExtensionTrait.scala +++ b/gluten-core/src/main/scala/io/glutenproject/extension/ExpressionExtensionTrait.scala @@ -20,6 +20,10 @@ import io.glutenproject.expression.{ExpressionTransformer, Sig} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, AggregateMode} +import org.apache.spark.sql.types.DataType + +import scala.collection.mutable.ListBuffer trait ExpressionExtensionTrait { @@ -34,21 +38,32 @@ trait ExpressionExtensionTrait { def replaceWithExtensionExpressionTransformer( substraitExprName: String, expr: Expression, - attributeSeq: Seq[Attribute]): ExpressionTransformer + attributeSeq: Seq[Attribute]): ExpressionTransformer = { + throw new UnsupportedOperationException(s"${expr.getClass} or $expr is not supported.") + } + + /** Get the attribute index of the extension aggregate functions. */ + def getAttrsIndexForExtensionAggregateExpr( + aggregateFunc: AggregateFunction, + mode: AggregateMode, + exp: AggregateExpression, + aggregateAttributeList: Seq[Attribute], + aggregateAttr: ListBuffer[Attribute], + resIndex: Int): Int = { + throw new UnsupportedOperationException( + s"Aggregate function ${aggregateFunc.getClass} is not supported.") + } + + /** Get the custom agg function substrait name and the input types of the child */ + def buildCustomAggregateFunction( + aggregateFunc: AggregateFunction): (Option[String], Seq[DataType]) = { + throw new UnsupportedOperationException( + s"Aggregate function ${aggregateFunc.getClass} is not supported.") + } } case class DefaultExpressionExtensionTransformer() extends ExpressionExtensionTrait with Logging { /** Generate the extension expressions list, format: Sig[XXXExpression]("XXXExpressionName") */ override def expressionSigList: Seq[Sig] = Seq.empty[Sig] - - /** Replace extension expression to transformer. */ - override def replaceWithExtensionExpressionTransformer( - substraitExprName: String, - expr: Expression, - attributeSeq: Seq[Attribute]): ExpressionTransformer = { - logWarning(s"${expr.getClass} or $expr is not currently supported.") - throw new UnsupportedOperationException( - s"${expr.getClass} or $expr is not currently supported.") - } } diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/RewriteDateTimestampComparisonRule.scala b/gluten-core/src/main/scala/io/glutenproject/extension/RewriteDateTimestampComparisonRule.scala new file mode 100644 index 000000000000..5bbc9fb1820f --- /dev/null +++ b/gluten-core/src/main/scala/io/glutenproject/extension/RewriteDateTimestampComparisonRule.scala @@ -0,0 +1,326 @@ +/* + * 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.extension + +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.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.lang.IllegalArgumentException + +// For readable, people usually convert a unix timestamp into date, and compare it with another +// date. For example +// select * from table where '2023-11-02' >= from_unixtime(unix_timestamp, 'yyyy-MM-dd') +// There are performance shortcomings +// 1. convert a unix timestamp into date is expensive +// 2. comparisoin with date or string is not efficient. +// +// This rule try to make the filter condition into integer comparison, which is more efficient. +// The above example will be rewritten into +// select * from table where to_unixtime('2023-11-02', 'yyyy-MM-dd') >= unix_timestamp +class RewriteDateTimestampComparisonRule(session: SparkSession, conf: SQLConf) + extends Rule[LogicalPlan] + with Logging { + + object TimeUnit extends Enumeration { + val SECOND, MINUTE, HOUR, DAY, MONTH, YEAR = Value + } + + override def apply(plan: LogicalPlan): LogicalPlan = { + if (plan.resolved) { + visitPlan(plan) + } else { + plan + } + } + + private def visitPlan(plan: LogicalPlan): LogicalPlan = plan match { + case filter: Filter => + val newCondition = visitExpression(filter.condition) + val newFilter = Filter(newCondition, filter.child) + newFilter + case other => + val children = other.children.map(visitPlan) + other.withNewChildren(children) + } + + private def visitExpression(expression: Expression): Expression = expression match { + case cmp: BinaryComparison => + if (isConstDateExpression(cmp.left) && isDateFromUnixTimestamp(cmp.right)) { + rewriteComparisionBetweenTimestampAndDate(exchangeOperators(cmp)) + } else if (isConstDateExpression(cmp.right) && isDateFromUnixTimestamp(cmp.left)) { + rewriteComparisionBetweenTimestampAndDate(cmp) + } else { + val children = expression.children.map(visitExpression) + expression.withNewChildren(children) + } + case literal: Literal => + expression + case _ => + val children = expression.children.map(visitExpression) + expression.withNewChildren(children) + } + + private def isConstDateExpression(expression: Expression): Boolean = { + def allConstExpression(expr: Expression): Boolean = expr match { + case literal: Literal => true + case attr: Attribute => false + case _ => expr.children.forall(allConstExpression) + } + if ( + !expression.dataType.isInstanceOf[DateType] && !expression.dataType.isInstanceOf[StringType] + ) { + return false + } + if (!allConstExpression(expression)) { + return false + } + true + } + + private def isDateFromUnixTimestamp(expr: Expression): Boolean = + expr match { + case toDate: ParseToDate => + isDateFromUnixTimestamp(toDate.left) + case fromUnixTime: FromUnixTime => + true + case _ => false + } + + private def getDateTimeUnit(expr: Expression): Option[TimeUnit.Value] = { + expr match { + case toDate: ParseToDate => + val timeUnit = if (toDate.format.isEmpty) { + Some(TimeUnit.DAY) + } else { + getDateTimeUnitFromLiteral(toDate.format) + } + val nestedTimeUnit = getDateTimeUnit(toDate.left) + if (nestedTimeUnit.isEmpty) { + timeUnit + } else { + if (nestedTimeUnit.get > timeUnit.get) { + nestedTimeUnit + } else { + timeUnit + } + } + case fromUnixTime: FromUnixTime => + getDateTimeUnitFromLiteral(Some(fromUnixTime.format)) + case _ => None + } + } + + private def getDateTimeUnitFromLiteral(expr: Option[Expression]): Option[TimeUnit.Value] = { + if (expr.isEmpty) { + Some(TimeUnit.SECOND) + } else if ( + !expr.get.isInstanceOf[Literal] || !expr.get + .asInstanceOf[Literal] + .dataType + .isInstanceOf[StringType] + ) { + None + } else { + val formatExpr = expr.get.asInstanceOf[Literal] + val formatStr = formatExpr.value.asInstanceOf[UTF8String].toString + if (formatStr.contains("ss")) { + Some(TimeUnit.SECOND) + } else if (formatStr.contains("mm")) { + Some(TimeUnit.MINUTE) + } else if (formatStr.contains("HH")) { + Some(TimeUnit.HOUR) + } else if (formatStr.contains("dd")) { + Some(TimeUnit.DAY) + } else if (formatStr.contains("MM")) { + Some(TimeUnit.MONTH) + } else if (formatStr.contains("yyyy")) { + Some(TimeUnit.YEAR) + } else { + None + } + } + } + + private def getTimeZoneId(expr: Expression): Option[String] = { + expr match { + case toDate: ParseToDate => + getTimeZoneId(toDate.left) + case fromUnixTime: FromUnixTime => + fromUnixTime.timeZoneId + case _ => None + } + } + + private def timeUnitToFormat(timeUnit: TimeUnit.Value): String = { + timeUnit match { + case TimeUnit.SECOND => "yyyy-MM-dd HH:mm:ss" + case TimeUnit.MINUTE => "yyyy-MM-dd HH:mm" + case TimeUnit.HOUR => "yyyy-MM-dd HH" + case TimeUnit.DAY => "yyyy-MM-dd" + case TimeUnit.MONTH => "yyyy-MM" + case TimeUnit.YEAR => "yyyy" + } + } + + private def rewriteConstDate( + expr: Expression, + timeUnit: TimeUnit.Value, + zoneId: Option[String], + adjustedOffset: Long): Expression = { + val formatExpr = Literal(UTF8String.fromString(timeUnitToFormat(timeUnit)), StringType) + val adjustExpr = Literal(adjustedOffset, LongType) + val toUnixTimestampExpr = ToUnixTimestamp(expr, formatExpr, zoneId) + Add(toUnixTimestampExpr, adjustExpr) + } + + private def rewriteUnixTimestampToDate(expr: Expression): Expression = { + expr match { + case toDate: ParseToDate => + rewriteUnixTimestampToDate(toDate.left) + case fromUnixTime: FromUnixTime => + fromUnixTime.sec + case _ => throw new IllegalArgumentException(s"Invalid expression: $expr") + } + } + + private def exchangeOperators(cmp: BinaryComparison): BinaryComparison = { + cmp match { + case gt: GreaterThan => + LessThan(cmp.right, cmp.left) + case gte: GreaterThanOrEqual => + LessThanOrEqual(cmp.right, cmp.left) + case lt: LessThan => + GreaterThan(cmp.right, cmp.left) + case lte: LessThanOrEqual => + GreaterThanOrEqual(cmp.right, cmp.left) + case eq: EqualTo => + EqualTo(cmp.right, cmp.left) + case eqn: EqualNullSafe => + EqualNullSafe(cmp.right, cmp.left) + } + } + + private def rewriteComparisionBetweenTimestampAndDate(cmp: BinaryComparison): Expression = { + val res = cmp match { + case gt: GreaterThan => + rewriteGreaterThen(gt) + case gte: GreaterThanOrEqual => + rewriteGreaterThanOrEqual(gte) + case lt: LessThan => + rewriteLessThen(lt) + case lte: LessThanOrEqual => + rewriteLessThenOrEqual(lte) + case eq: EqualTo => + rewriteEqualTo(eq) + case eqn: EqualNullSafe => + rewriteEqualNullSafe(eqn) + } + logInfo(s"rewrite expresion $cmp to $res") + res + } + + def TimeUnitToSeconds(timeUnit: TimeUnit.Value): Long = timeUnit match { + case TimeUnit.SECOND => 1 + case TimeUnit.MINUTE => 60 + case TimeUnit.HOUR => 3600 + case TimeUnit.DAY => 86400 + case TimeUnit.MONTH => 2592000 + case TimeUnit.YEAR => 31536000 + } + + private def rewriteGreaterThen(cmp: GreaterThan): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val adjustedOffset = TimeUnitToSeconds(timeUnit.get) + val newLeft = rewriteUnixTimestampToDate(cmp.left) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, adjustedOffset) + GreaterThanOrEqual(newLeft, newRight) + } + + private def rewriteGreaterThanOrEqual(cmp: GreaterThanOrEqual): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val adjustedOffset = 0 + val newLeft = rewriteUnixTimestampToDate(cmp.left) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, adjustedOffset) + GreaterThanOrEqual(newLeft, newRight) + } + + private def rewriteLessThen(cmp: LessThan): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val adjustedOffset = 0 + val newLeft = rewriteUnixTimestampToDate(cmp.left) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, adjustedOffset) + LessThan(newLeft, newRight) + } + + private def rewriteLessThenOrEqual(cmp: LessThanOrEqual): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val adjustedOffset = TimeUnitToSeconds(timeUnit.get) + val newLeft = rewriteUnixTimestampToDate(cmp.left) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, adjustedOffset) + LessThan(newLeft, newRight) + } + + private def rewriteEqualTo(cmp: EqualTo): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val timestampLeft = rewriteUnixTimestampToDate(cmp.left) + val adjustedOffset = Literal(TimeUnitToSeconds(timeUnit.get), timestampLeft.dataType) + val addjustedOffsetExpr = Remainder(timestampLeft, adjustedOffset) + val newLeft = Subtract(timestampLeft, addjustedOffsetExpr) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, 0) + EqualTo(newLeft, newRight) + } + + private def rewriteEqualNullSafe(cmp: EqualNullSafe): Expression = { + val timeUnit = getDateTimeUnit(cmp.left) + if (timeUnit.isEmpty) { + return cmp + } + val zoneId = getTimeZoneId(cmp.left) + val timestampLeft = rewriteUnixTimestampToDate(cmp.left) + val adjustedOffset = Literal(TimeUnitToSeconds(timeUnit.get), timestampLeft.dataType) + val addjustedOffsetExpr = Remainder(timestampLeft, adjustedOffset) + val newLeft = Subtract(timestampLeft, addjustedOffsetExpr) + val newRight = rewriteConstDate(cmp.right, timeUnit.get, zoneId, 0) + EqualNullSafe(newLeft, newRight) + } +} diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/RewriteTransformer.scala b/gluten-core/src/main/scala/io/glutenproject/extension/RewriteTransformer.scala new file mode 100644 index 000000000000..ad8d7cd046fc --- /dev/null +++ b/gluten-core/src/main/scala/io/glutenproject/extension/RewriteTransformer.scala @@ -0,0 +1,53 @@ +/* + * 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 io.glutenproject.extension + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ + +trait RewriteTransformerRules { + def rules: Seq[Rule[SparkPlan]] +} + +case class RewriteTransformer(session: SparkSession) extends Rule[SparkPlan] { + + private val rules: Seq[Rule[SparkPlan]] = RewriteTransformer.loadRewritePlanRules + + override def apply(plan: SparkPlan): SparkPlan = { + rules.foldLeft(plan) { + case (plan, rule) => + rule(plan) + } + } + +} + +object RewriteTransformer { + + private def loadRewritePlanRules: Seq[Rule[SparkPlan]] = { + val loader = Option(Thread.currentThread().getContextClassLoader) + .getOrElse(getClass.getClassLoader) + val serviceLoader = ServiceLoader.load(classOf[RewriteTransformerRules], loader) + + serviceLoader.asScala.flatMap(_.rules).toSeq + } +} diff --git a/gluten-core/src/main/scala/io/glutenproject/extension/columnar/TransformHintRule.scala b/gluten-core/src/main/scala/io/glutenproject/extension/columnar/TransformHintRule.scala index 7bdb8157bc7f..7bfcd35382c3 100644 --- a/gluten-core/src/main/scala/io/glutenproject/extension/columnar/TransformHintRule.scala +++ b/gluten-core/src/main/scala/io/glutenproject/extension/columnar/TransformHintRule.scala @@ -20,12 +20,13 @@ import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution._ import io.glutenproject.extension.{GlutenPlan, ValidationResult} +import io.glutenproject.sql.shims.SparkShimLoader import io.glutenproject.utils.PhysicalPlanSelector import org.apache.spark.api.python.EvalPythonExecTransformer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SortOrder} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} import org.apache.spark.sql.catalyst.plans.FullOuter import org.apache.spark.sql.catalyst.rules.Rule @@ -312,8 +313,10 @@ case class AddTransformHintRule() extends Rule[SparkPlan] { /** Inserts a transformable tag on top of those that are not supported. */ private def addTransformableTags(plan: SparkPlan): SparkPlan = { - addTransformableTag(plan) - plan.withNewChildren(plan.children.map(addTransformableTags)) + // Walk the tree with post-order + val out = plan.withNewChildren(plan.children.map(addTransformableTags)) + addTransformableTag(out) + out } private def addTransformableTag(plan: SparkPlan): Unit = { @@ -333,8 +336,11 @@ case class AddTransformHintRule() extends Rule[SparkPlan] { if (plan.runtimeFilters.nonEmpty) { TransformHints.tagTransformable(plan) } else { - val transformer = - new BatchScanExecTransformer(plan.output, plan.scan, plan.runtimeFilters) + val transformer = new BatchScanExecTransformer( + plan.output, + plan.scan, + plan.runtimeFilters, + table = SparkShimLoader.getSparkShims.getBatchScanExecTable(plan)) TransformHints.tag(plan, transformer.doValidate().toTransformHint) } } @@ -547,7 +553,9 @@ case class AddTransformHintRule() extends Rule[SparkPlan] { maybeExchange match { case Some(exchange @ BroadcastExchangeExec(mode, child)) => TransformHints.tag(bhj, isBhjTransformable.toTransformHint) - TransformHints.tagNotTransformable(exchange, isBhjTransformable) + if (!isBhjTransformable.isValid) { + TransformHints.tagNotTransformable(exchange, isBhjTransformable) + } case None => // we are in AQE, find the hidden exchange // FIXME did we consider the case that AQE: OFF && Reuse: ON ? @@ -673,12 +681,17 @@ case class AddTransformHintRule() extends Rule[SparkPlan] { "columnar topK is not enabled in TakeOrderedAndProjectExec") } else { var tagged: ValidationResult = null - val limitPlan = LimitTransformer(plan.child, 0, plan.limit) - tagged = limitPlan.doValidate() - if (tagged.isValid) { + val orderingSatisfies = + SortOrder.orderingSatisfies(plan.child.outputOrdering, plan.sortOrder) + if (orderingSatisfies) { + val limitPlan = LimitTransformer(plan.child, 0, plan.limit) + tagged = limitPlan.doValidate() + } else { val sortPlan = SortExecTransformer(plan.sortOrder, false, plan.child) - tagged = sortPlan.doValidate() + val limitPlan = LimitTransformer(sortPlan, 0, plan.limit) + tagged = limitPlan.doValidate() } + if (tagged.isValid) { val projectPlan = ProjectExecTransformer(plan.projectList, plan.child) tagged = projectPlan.doValidate() diff --git a/gluten-core/src/main/scala/io/glutenproject/metrics/MetricsUpdater.scala b/gluten-core/src/main/scala/io/glutenproject/metrics/MetricsUpdater.scala index 1aa3616331a9..030241757419 100644 --- a/gluten-core/src/main/scala/io/glutenproject/metrics/MetricsUpdater.scala +++ b/gluten-core/src/main/scala/io/glutenproject/metrics/MetricsUpdater.scala @@ -36,6 +36,6 @@ trait MetricsUpdater extends Serializable { final case class MetricsUpdaterTree(updater: MetricsUpdater, children: Seq[MetricsUpdaterTree]) -class NoopMetricsUpdater extends MetricsUpdater { +object NoopMetricsUpdater extends MetricsUpdater { override def metrics: Map[String, SQLMetric] = Map.empty } diff --git a/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala b/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala index b1cacd354d33..2a2bae1413cb 100644 --- a/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala +++ b/gluten-core/src/main/scala/io/glutenproject/substrait/SubstraitContext.scala @@ -17,12 +17,12 @@ package io.glutenproject.substrait import io.glutenproject.substrait.ddlplan.InsertOutputNode -import io.glutenproject.substrait.rel.LocalFilesNode +import io.glutenproject.substrait.rel.{LocalFilesNode, SplitInfo} import io.glutenproject.substrait.rel.LocalFilesNode.ReadFileFormat import java.lang.{Integer => JInt, Long => JLong} import java.security.InvalidParameterException -import java.util.{ArrayList => JArrayList, HashMap => JHashMap, List => JList} +import java.util.{ArrayList => JArrayList, HashMap => JHashMap, List => JList, Map => JMap} case class JoinParams() { // Whether the input of streamed side is a ReadRel represented iterator. @@ -69,7 +69,7 @@ class SubstraitContext extends Serializable { private val iteratorNodes = new JHashMap[JLong, LocalFilesNode]() // A map stores the relationship between Spark operator id and its respective Substrait Rel ids. - private val operatorToRelsMap = new JHashMap[JLong, JArrayList[JLong]]() + private val operatorToRelsMap: JMap[JLong, JList[JLong]] = new JHashMap[JLong, JList[JLong]]() // Only for debug conveniently private val operatorToPlanNameMap = new JHashMap[JLong, String]() @@ -80,8 +80,8 @@ class SubstraitContext extends Serializable { // A map stores the relationship between aggregation operator id and its param. private val aggregationParamsMap = new JHashMap[JLong, AggregationParams]() - private var localFilesNodesIndex: JInt = 0 - private var localFilesNodes: Seq[java.io.Serializable] = _ + private var splitInfosIndex: JInt = 0 + private var splitInfos: Seq[SplitInfo] = _ private var iteratorIndex: JLong = 0L private var fileFormat: JList[ReadFileFormat] = new JArrayList[ReadFileFormat]() private var insertOutputNode: InsertOutputNode = _ @@ -95,28 +95,28 @@ class SubstraitContext extends Serializable { iteratorNodes.put(index, localFilesNode) } - def initLocalFilesNodesIndex(localFilesNodesIndex: JInt): Unit = { - this.localFilesNodesIndex = localFilesNodesIndex + def initSplitInfosIndex(splitInfosIndex: JInt): Unit = { + this.splitInfosIndex = splitInfosIndex } - def getLocalFilesNodes: Seq[java.io.Serializable] = localFilesNodes + def getSplitInfos: Seq[SplitInfo] = splitInfos // FIXME Hongze 22/11/28 // This makes calls to ReadRelNode#toProtobuf non-idempotent which doesn't seem to be // optimal in regard to the method name "toProtobuf". - def getCurrentLocalFileNode: java.io.Serializable = { - if (getLocalFilesNodes != null && getLocalFilesNodes.size > localFilesNodesIndex) { - val res = getLocalFilesNodes(localFilesNodesIndex) - localFilesNodesIndex += 1 + def getCurrentSplitInfo: SplitInfo = { + if (getSplitInfos != null && getSplitInfos.size > splitInfosIndex) { + val res = getSplitInfos(splitInfosIndex) + splitInfosIndex += 1 res } else { throw new IllegalStateException( - s"LocalFilesNodes index $localFilesNodesIndex exceeds the size of the LocalFilesNodes.") + s"LocalFilesNodes index $splitInfosIndex exceeds the size of the LocalFilesNodes.") } } - def setLocalFilesNodes(localFilesNodes: Seq[java.io.Serializable]): Unit = { - this.localFilesNodes = localFilesNodes + def setSplitInfos(SplitInfos: Seq[SplitInfo]): Unit = { + this.splitInfos = SplitInfos } def getInputIteratorNode(index: JLong): LocalFilesNode = { @@ -164,6 +164,25 @@ class SubstraitContext extends Serializable { relId += 1 } + /** Register a specified rel to certain operator id. */ + def registerRelToOperator(operatorId: JLong, specifiedRedId: JLong): Unit = { + if (operatorToRelsMap.containsKey(operatorId)) { + val rels = operatorToRelsMap.get(operatorId) + rels.add(specifiedRedId) + } else { + val rels = new JArrayList[JLong]() + rels.add(specifiedRedId) + operatorToRelsMap.put(operatorId, rels) + } + } + + /** Add the relId and register to operator later */ + def nextRelId(): JLong = { + val id = this.relId + this.relId += 1 + id + } + /** * Register empty rel list to certain operator id. Used when the computing of a Spark transformer * is omitted. @@ -181,7 +200,7 @@ class SubstraitContext extends Serializable { * Return the registered map. * @return */ - def registeredRelMap: JHashMap[JLong, JArrayList[JLong]] = operatorToRelsMap + def registeredRelMap: JMap[JLong, JList[JLong]] = operatorToRelsMap /** * Register the join params to certain operator id. diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/DebugUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/DebugUtil.scala index 868b843feed5..89010f232e5e 100644 --- a/gluten-core/src/main/scala/io/glutenproject/utils/DebugUtil.scala +++ b/gluten-core/src/main/scala/io/glutenproject/utils/DebugUtil.scala @@ -21,23 +21,28 @@ import io.glutenproject.GlutenConfig import org.apache.spark.TaskContext object DebugUtil { - // if not specify stageId, use default 1 - // if specify partitionId, use this one - // if not, use specified taskId or save all the input of this stage + // if not in debug mode, then do nothing + // if specify taskId, then only do that task partition + // if not specify stageId, 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 (!GlutenConfig.getConf.debug) { return false } + 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) { - true - } else if (GlutenConfig.getConf.taskPartitionId == -1) { - TaskContext.get().taskAttemptId() == GlutenConfig.getConf.taskId || - GlutenConfig.getConf.taskId == -1 - } else { - false + return true } + + false } } diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/DecimalArithmeticUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/DecimalArithmeticUtil.scala new file mode 100644 index 000000000000..da1feab1e570 --- /dev/null +++ b/gluten-core/src/main/scala/io/glutenproject/utils/DecimalArithmeticUtil.scala @@ -0,0 +1,291 @@ +/* + * 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 io.glutenproject.utils + +import io.glutenproject.backendsapi.BackendsApiManager +import io.glutenproject.expression.{CheckOverflowTransformer, ChildTransformer, DecimalArithmeticExpressionTransformer, ExpressionTransformer} + +import org.apache.spark.sql.catalyst.analysis.DecimalPrecision +import org.apache.spark.sql.catalyst.expressions.{Add, BinaryArithmetic, Cast, Divide, Expression, Literal, Multiply, Pmod, PromotePrecision, Remainder, Subtract} +import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, IntegerType, LongType, ShortType} + +object DecimalArithmeticUtil { + + object OperationType extends Enumeration { + type Config = Value + val ADD, SUBTRACT, MULTIPLY, DIVIDE, MOD = Value + } + + val MIN_ADJUSTED_SCALE = 6 + val MAX_PRECISION = 38 + + // Returns the result decimal type of a decimal arithmetic computing. + def getResultTypeForOperation( + operationType: OperationType.Config, + type1: DecimalType, + type2: DecimalType): DecimalType = { + var resultScale = 0 + var resultPrecision = 0 + operationType match { + case OperationType.ADD => + resultScale = Math.max(type1.scale, type2.scale) + resultPrecision = + resultScale + Math.max(type1.precision - type1.scale, type2.precision - type2.scale) + 1 + case OperationType.SUBTRACT => + resultScale = Math.max(type1.scale, type2.scale) + resultPrecision = + resultScale + Math.max(type1.precision - type1.scale, type2.precision - type2.scale) + 1 + case OperationType.MULTIPLY => + resultScale = type1.scale + type2.scale + resultPrecision = type1.precision + type2.precision + 1 + case OperationType.DIVIDE => + resultScale = Math.max(MIN_ADJUSTED_SCALE, type1.scale + type2.precision + 1) + resultPrecision = type1.precision - type1.scale + type2.scale + resultScale + case OperationType.MOD => + resultScale = Math.max(type1.scale, type2.scale) + resultPrecision = + Math.min(type1.precision - type1.scale, type2.precision - type2.scale + resultScale) + case other => + throw new UnsupportedOperationException(s"$other is not supported.") + } + adjustScaleIfNeeded(resultPrecision, resultScale) + } + + // Returns the adjusted decimal type when the precision is larger the maximum. + def adjustScaleIfNeeded(precision: Int, scale: Int): DecimalType = { + var typePrecision = precision + var typeScale = scale + if (precision > MAX_PRECISION) { + val minScale = Math.min(scale, MIN_ADJUSTED_SCALE) + val delta = precision - MAX_PRECISION + typePrecision = MAX_PRECISION + typeScale = Math.max(scale - delta, minScale) + } + DecimalType(typePrecision, typeScale) + } + + // If casting between DecimalType, unnecessary cast is skipped to avoid data loss, + // because argument input type of "cast" is actually the res type of "+-*/". + // Cast will use a wider input type, then calculates result type with less scale than expected. + def isDecimalArithmetic(b: BinaryArithmetic): Boolean = { + if ( + b.left.dataType.isInstanceOf[DecimalType] && + b.right.dataType.isInstanceOf[DecimalType] + ) { + b match { + case _: Divide | _: Multiply | _: Add | _: Subtract | _: Remainder | _: Pmod => true + case _ => false + } + } else false + } + + // Returns the operation type of a binary arithmetic expression. + def getOperationType(b: BinaryArithmetic): OperationType.Config = { + b match { + case _: Add => OperationType.ADD + case _: Subtract => OperationType.SUBTRACT + case _: Multiply => OperationType.MULTIPLY + case _: Divide => OperationType.DIVIDE + case other => + throw new UnsupportedOperationException(s"$other is not supported.") + } + } + + // For decimal * 10 case, dec will be Decimal(38, 18), then the result precision is wrong, + // so here we will get the real precision and scale of the literal. + private def getNewPrecisionScale(dec: Decimal): (Integer, Integer) = { + val input = dec.abs.toString() + val dotIndex = input.indexOf(".") + if (dotIndex == -1) { + return (input.length, 0) + } + if (dec.toBigDecimal.isValidLong) { + return (dotIndex, 0) + } + (dec.precision, dec.scale) + } + + // Change the precision and scale to the actual precision and scale of a literal, + // otherwise the result precision loses. + def rescaleLiteral(arithmeticExpr: BinaryArithmetic): BinaryArithmetic = { + if ( + arithmeticExpr.left.isInstanceOf[PromotePrecision] && + arithmeticExpr.right.isInstanceOf[Literal] + ) { + val lit = arithmeticExpr.right.asInstanceOf[Literal] + lit.value match { + case decLit: Decimal => + val (precision, scale) = getNewPrecisionScale(decLit) + if (precision != decLit.precision || scale != decLit.scale) { + arithmeticExpr + .withNewChildren(Seq(arithmeticExpr.left, Cast(lit, DecimalType(precision, scale)))) + .asInstanceOf[BinaryArithmetic] + } else arithmeticExpr + case _ => arithmeticExpr + } + } else if ( + arithmeticExpr.right.isInstanceOf[PromotePrecision] + && arithmeticExpr.left.isInstanceOf[Literal] + ) { + val lit = arithmeticExpr.left.asInstanceOf[Literal] + lit.value match { + case decLit: Decimal => + val (precision, scale) = getNewPrecisionScale(decLit) + if (precision != decLit.precision || scale != decLit.scale) { + arithmeticExpr + .withNewChildren(Seq(Cast(lit, DecimalType(precision, scale)), arithmeticExpr.right)) + .asInstanceOf[BinaryArithmetic] + } else arithmeticExpr + case _ => arithmeticExpr + } + } else { + arithmeticExpr + } + } + + // Returns whether the input expression is a combination of PromotePrecision(Cast as DecimalType). + private def isPromoteCast(expr: Expression): Boolean = { + expr match { + case precision: PromotePrecision => + precision.child match { + case cast: Cast if cast.dataType.isInstanceOf[DecimalType] => true + case _ => false + } + case _ => false + } + } + + def rescaleCastForDecimal(left: Expression, right: Expression): (Expression, Expression) = { + if (!BackendsApiManager.getSettings.rescaleDecimalIntegralExpression()) { + return (left, right) + } + // Decimal * cast int. + if (!isPromoteCast(left)) { + // Have removed PromotePrecision(Cast(DecimalType)). + if (isPromoteCastIntegral(right)) { + val newRight = rescaleCastForOneSide(right) + val isWiderType = checkIsWiderType( + left.dataType.asInstanceOf[DecimalType], + newRight.dataType.asInstanceOf[DecimalType], + right.dataType.asInstanceOf[DecimalType]) + if (isWiderType) { + (left, newRight) + } else { + (left, right) + } + } else { + (left, right) + } + // Cast int * decimal. + } else if (!isPromoteCast(right)) { + if (isPromoteCastIntegral(left)) { + val newLeft = rescaleCastForOneSide(left) + val isWiderType = checkIsWiderType( + newLeft.dataType.asInstanceOf[DecimalType], + right.dataType.asInstanceOf[DecimalType], + left.dataType.asInstanceOf[DecimalType]) + if (isWiderType) { + (newLeft, right) + } else { + (left, right) + } + } else { + (left, right) + } + } else { + // Cast int * cast int. Usually user defined cast. + (left, right) + } + } + + /** + * Remove the Cast when child is PromotePrecision and PromotePrecision is Cast(Decimal, Decimal) + * + * @param arithmeticExpr + * BinaryArithmetic left or right + * @return + * expression removed child PromotePrecision->Cast + */ + def removeCastForDecimal(arithmeticExpr: Expression): Expression = { + arithmeticExpr match { + case precision: PromotePrecision => + precision.child match { + case cast: Cast + if cast.dataType.isInstanceOf[DecimalType] + && cast.child.dataType.isInstanceOf[DecimalType] => + cast.child + case _ => arithmeticExpr + } + case _ => arithmeticExpr + } + } + + def getResultType(transformer: ExpressionTransformer): Option[DecimalType] = { + transformer match { + case ChildTransformer(child) => + getResultType(child) + case CheckOverflowTransformer(_, _, original) => + Some(original.dataType) + case DecimalArithmeticExpressionTransformer(_, _, _, resultType, _) => + Some(resultType) + case _ => None + } + } + + private def isPromoteCastIntegral(expr: Expression): Boolean = { + expr match { + case precision: PromotePrecision => + precision.child match { + case cast: Cast if cast.dataType.isInstanceOf[DecimalType] => + cast.child.dataType match { + case IntegerType | ByteType | ShortType | LongType => true + case _ => false + } + case _ => false + } + case _ => false + } + } + + private def rescaleCastForOneSide(expr: Expression): Expression = { + expr match { + case precision: PromotePrecision => + precision.child match { + case castInt: Cast + if castInt.dataType.isInstanceOf[DecimalType] && + BackendsApiManager.getSettings.rescaleDecimalIntegralExpression() => + castInt.child.dataType match { + case IntegerType | ByteType | ShortType => + precision.withNewChildren(Seq(Cast(castInt.child, DecimalType(10, 0)))) + case LongType => + precision.withNewChildren(Seq(Cast(castInt.child, DecimalType(20, 0)))) + case _ => expr + } + case _ => expr + } + case _ => expr + } + } + + private def checkIsWiderType( + left: DecimalType, + right: DecimalType, + wider: DecimalType): Boolean = { + val widerType = DecimalPrecision.widerDecimalType(left, right) + widerType.equals(wider) + } +} diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/FallbackUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/FallbackUtil.scala index 1502f94bca91..40c0786599f2 100644 --- a/gluten-core/src/main/scala/io/glutenproject/utils/FallbackUtil.scala +++ b/gluten-core/src/main/scala/io/glutenproject/utils/FallbackUtil.scala @@ -20,7 +20,7 @@ import io.glutenproject.extension.GlutenPlan import org.apache.spark.internal.Logging import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, ColumnarAQEShuffleReadExec, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec /** @@ -57,7 +57,7 @@ object FallbackUtil extends Logging with AdaptiveSparkPlanHelper { true case _: ReusedExchangeExec => true - case _: ColumnarAQEShuffleReadExec => + case p: SparkPlan if p.supportsColumnar => true case _ => false diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/GlutenDecimalUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/GlutenDecimalUtil.scala deleted file mode 100644 index e35473835d78..000000000000 --- a/gluten-core/src/main/scala/io/glutenproject/utils/GlutenDecimalUtil.scala +++ /dev/null @@ -1,40 +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 io.glutenproject.utils - -import org.apache.spark.sql.catalyst.expressions.aggregate.Average -import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType} -import org.apache.spark.sql.types.DecimalType.{MAX_PRECISION, MAX_SCALE} - -import scala.math.min - -object GlutenDecimalUtil { - object Fixed { - def unapply(t: DecimalType): Option[(Int, Int)] = Some((t.precision, t.scale)) - } - - def bounded(precision: Int, scale: Int): DecimalType = { - DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) - } - - def getAvgSumDataType(avg: Average): DataType = avg.dataType match { - // avg.dataType is Decimal(p + 4, s + 4) and sumType is Decimal(p + 10, s) - // we need to get sumType, so p = p - 4 + 10 and s = s - 4 - case _ @GlutenDecimalUtil.Fixed(p, s) => GlutenDecimalUtil.bounded(p - 4 + 10, s - 4) - case _ => DoubleType - } -} diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala b/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala index 7fbb6beb746d..37ee1eaf1fc7 100644 --- a/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala +++ b/gluten-core/src/main/scala/io/glutenproject/utils/InputPartitionsUtil.scala @@ -29,24 +29,12 @@ case class InputPartitionsUtil( relation: HadoopFsRelation, selectedPartitions: Array[PartitionDirectory], output: Seq[Attribute], + bucketedScan: Boolean, optionalBucketSet: Option[BitSet], optionalNumCoalescedBuckets: Option[Int], disableBucketedScan: Boolean) extends Logging { - private val bucketedScan: Boolean = { - if ( - relation.sparkSession.sessionState.conf.bucketingEnabled && relation.bucketSpec.isDefined - && !disableBucketedScan - ) { - val spec = relation.bucketSpec.get - val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) - bucketColumns.size == spec.bucketColumnNames.size - } else { - false - } - } - def genInputPartitionSeq(): Seq[InputPartition] = { if (bucketedScan) { genBucketedInputPartitionSeq() diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/Iterators.scala b/gluten-core/src/main/scala/io/glutenproject/utils/Iterators.scala new file mode 100644 index 000000000000..916a0d2a7e9d --- /dev/null +++ b/gluten-core/src/main/scala/io/glutenproject/utils/Iterators.scala @@ -0,0 +1,204 @@ +/* + * 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 io.glutenproject.utils + +import org.apache.spark.{InterruptibleIterator, TaskContext} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.util.TaskResources + +import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} + +private class PayloadCloser[A](in: Iterator[A])(closeCallback: A => Unit) extends Iterator[A] { + private var closer: Option[() => Unit] = None + + TaskResources.addRecycler("Iterators#PayloadCloser", 100) { + tryClose() + } + + override def hasNext: Boolean = { + tryClose() + in.hasNext + } + + override def next(): A = { + val a: A = in.next() + closer.synchronized { + closer = Some( + () => { + closeCallback.apply(a) + }) + } + a + } + + private def tryClose(): Unit = { + closer.synchronized { + closer match { + case Some(c) => c.apply() + case None => + } + closer = None // make sure the payload is closed once + } + } +} + +private class IteratorCompleter[A](in: Iterator[A])(completionCallback: => Unit) + extends Iterator[A] { + private val completed = new AtomicBoolean(false) + + TaskResources.addRecycler("Iterators#IteratorRecycler", 100) { + tryComplete() + } + + override def hasNext: Boolean = { + val out = in.hasNext + if (!out) { + tryComplete() + } + out + } + + override def next(): A = { + in.next() + } + + private def tryComplete(): Unit = { + if (!completed.compareAndSet(false, true)) { + return // make sure the iterator is completed once + } + completionCallback + } +} + +private class PipelineTimeAccumulator[A](in: Iterator[A], pipelineTime: SQLMetric) + extends Iterator[A] { + private val accumulatedTime: AtomicLong = new AtomicLong(0L) + + TaskResources.addRecycler("Iterators#PipelineTimeAccumulator", 100) { + tryFinish() + } + + override def hasNext: Boolean = { + val prev = System.nanoTime() + val out = in.hasNext + accumulatedTime.addAndGet(System.nanoTime() - prev) + if (!out) { + tryFinish() + } + out + } + + override def next(): A = { + val prev = System.nanoTime() + val out = in.next() + accumulatedTime.addAndGet(System.nanoTime() - prev) + out + } + + private def tryFinish(): Unit = { + pipelineTime += TimeUnit.NANOSECONDS.toMillis( + accumulatedTime.getAndSet( + 0L + ) + ) // make sure the accumulated time is submitted once + } +} + +/** + * To protect the wrapped iterator to avoid undesired order of calls to its `hasNext` and `next` + * methods. + */ +private class InvocationFlowProtection[A](in: Iterator[A]) extends Iterator[A] { + sealed private trait State; + private case object Init extends State; + private case class HasNextCalled(hasNext: Boolean) extends State; + private case object NextCalled extends State; + + private var state: State = Init + + override def hasNext: Boolean = { + val out = state match { + case Init | NextCalled => + in.hasNext + case HasNextCalled(lastHasNext) => + lastHasNext + } + state = HasNextCalled(out) + out + } + + override def next(): A = { + val out = state match { + case Init | NextCalled => + if (!in.hasNext) { + throw new IllegalStateException("End of stream") + } + in.next() + case HasNextCalled(lastHasNext) => + if (!lastHasNext) { + throw new IllegalStateException("End of stream") + } + in.next() + } + state = NextCalled + out + } +} + +class WrapperBuilder[A](in: Iterator[A]) { // FIXME how to make the ctor companion-private? + private var wrapped: Iterator[A] = in + + def recyclePayload(closeCallback: (A) => Unit): WrapperBuilder[A] = { + wrapped = new PayloadCloser(wrapped)(closeCallback) + this + } + + def recycleIterator(completionCallback: => Unit): WrapperBuilder[A] = { + wrapped = new IteratorCompleter(wrapped)(completionCallback) + this + } + + def addToPipelineTime(pipelineTime: SQLMetric): WrapperBuilder[A] = { + wrapped = new PipelineTimeAccumulator[A](wrapped, pipelineTime) + this + } + + def asInterruptible(context: TaskContext): WrapperBuilder[A] = { + wrapped = new InterruptibleIterator[A](context, wrapped) + this + } + + def protectInvocationFlow(): WrapperBuilder[A] = { + wrapped = new InvocationFlowProtection[A](wrapped) + this + } + + def create(): Iterator[A] = { + wrapped + } +} + +/** + * Utility class to provide iterator wrappers for non-trivial use cases. E.g. iterators that manage + * payload's lifecycle. + */ +object Iterators { + def wrap[A](in: Iterator[A]): WrapperBuilder[A] = { + new WrapperBuilder[A](in) + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala b/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala index aa70571850b1..8ae9197a212f 100644 --- a/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/memory/SparkMemoryUtil.scala @@ -58,6 +58,7 @@ object SparkMemoryUtil { overAcquire.getTarget.accept(this) } + @scala.annotation.nowarn override def visit(regularMemoryConsumer: RegularMemoryConsumer): KnownNameAndStats = { collectFromTaskMemoryManager(regularMemoryConsumer.getTaskMemoryManager) } diff --git a/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala b/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala index b5922fab3ec4..34492be2e31e 100644 --- a/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala +++ b/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenDriverEndpoint.scala @@ -40,6 +40,8 @@ class GlutenDriverEndpoint extends IsolatedRpcEndpoint with Logging { private val driverEndpoint: RpcEndpointRef = rpcEnv.setupEndpoint(GlutenRpcConstants.GLUTEN_DRIVER_ENDPOINT_NAME, this) + // TODO(yuan): get thread cnt from spark context + override def threadCount(): Int = 1 override def receive: PartialFunction[Any, Unit] = { case GlutenOnExecutionStart(executionId) => if (executionId == null) { diff --git a/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala b/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala index fe2aed750751..2876c4dda4fb 100644 --- a/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala +++ b/gluten-core/src/main/scala/org/apache/spark/rpc/GlutenExecutorEndpoint.scala @@ -40,7 +40,8 @@ class GlutenExecutorEndpoint(val executorId: String, val conf: SparkConf) @volatile var driverEndpointRef: RpcEndpointRef = null rpcEnv.setupEndpoint(GlutenRpcConstants.GLUTEN_EXECUTOR_ENDPOINT_NAME, this) - + // TODO(yuan): get thread cnt from spark context + override def threadCount(): Int = 1 override def onStart(): Unit = { rpcEnv .asyncSetupEndpointRefByURI(driverUrl) diff --git a/gluten-core/src/main/scala/org/apache/spark/softaffinity/SoftAffinityUtil.scala b/gluten-core/src/main/scala/org/apache/spark/softaffinity/SoftAffinityUtil.scala index a6a3d75c76ca..3156991d6be9 100644 --- a/gluten-core/src/main/scala/org/apache/spark/softaffinity/SoftAffinityUtil.scala +++ b/gluten-core/src/main/scala/org/apache/spark/softaffinity/SoftAffinityUtil.scala @@ -23,42 +23,40 @@ import io.glutenproject.utils.LogLevelUtil import org.apache.spark.internal.Logging import org.apache.spark.scheduler.ExecutorCacheTaskLocation -import org.apache.spark.sql.execution.datasources.FilePartition object SoftAffinityUtil extends LogLevelUtil with Logging { private lazy val softAffinityLogLevel = GlutenConfig.getConf.softAffinityLogLevel /** Get the locations by SoftAffinityManager */ - def getFilePartitionLocations(filePartition: FilePartition): Array[String] = { - // Get the original preferred locations - val expectedTargets = filePartition.preferredLocations() - + def getFilePartitionLocations( + filePaths: Array[String], + preferredLocations: Array[String]): Array[String] = { if ( - !filePartition.files.isEmpty && SoftAffinityManager.usingSoftAffinity - && !SoftAffinityManager.checkTargetHosts(expectedTargets) + !filePaths.isEmpty && SoftAffinityManager.usingSoftAffinity + && !SoftAffinityManager.checkTargetHosts(preferredLocations) ) { // if there is no host in the node list which are executors running on, // using SoftAffinityManager to generate target executors. // Only using the first file to calculate the target executors // Only get one file to calculate the target host - val file = filePartition.files.sortBy(_.filePath).head - val locations = SoftAffinityManager.askExecutors(file.filePath) + val filePath = filePaths.min + val locations = SoftAffinityManager.askExecutors(filePath) if (!locations.isEmpty) { logOnLevel( softAffinityLogLevel, - s"SAMetrics=File ${file.filePath} - " + + s"SAMetrics=File $filePath - " + s"the expected executors are ${locations.mkString("_")} ") locations.map { p => if (p._1.equals("")) p._2 else ExecutorCacheTaskLocation(p._2, p._1).toString - }.toArray + } } else { Array.empty[String] } } else { - expectedTargets + preferredLocations } } @@ -77,7 +75,7 @@ object SoftAffinityUtil extends LogLevelUtil with Logging { p => if (p._1.equals("")) p._2 else ExecutorCacheTaskLocation(p._2, p._1).toString - }.toArray + } } else { Array.empty[String] } diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala index af6ee8712e18..3b8a72f1c18b 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/ColumnarShuffleExchangeExec.scala @@ -129,7 +129,7 @@ case class ColumnarShuffleExchangeExec( } override def getShuffleRDD(partitionSpecs: Array[ShufflePartitionSpec]): RDD[ColumnarBatch] = { - cachedShuffleRDD + new ShuffledColumnarBatchRDD(columnarShuffleDependency, readMetrics, partitionSpecs) } override def stringArgs: Iterator[Any] = @@ -165,6 +165,18 @@ case class ColumnarShuffleExchangeExec( } object ColumnarShuffleExchangeExec extends Logging { + + def apply( + plan: ShuffleExchangeExec, + child: SparkPlan, + shuffleOutputAttributes: Seq[Attribute]): ColumnarShuffleExchangeExec = { + ColumnarShuffleExchangeExec( + plan.outputPartitioning, + child, + plan.shuffleOrigin, + shuffleOutputAttributes) + } + // scalastyle:off argcount def prepareShuffleDependency( rdd: RDD[ColumnarBatch], diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala index e281d7f36f82..057786f73f50 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenExplainUtils.scala @@ -23,11 +23,13 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.execution.GlutenFallbackReporter.FALLBACK_REASON_TAG -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, QueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, AQEShuffleReadExec, 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.v2.V2CommandExec import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} -import java.util.{IdentityHashMap, Set} +import java.util import java.util.Collections.newSetFromMap import scala.collection.mutable @@ -38,25 +40,52 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} // 2. remove `plan.verboseStringWithOperatorId` // 3. remove codegen id object GlutenExplainUtils extends AdaptiveSparkPlanHelper { - private def collectFallbackNodes(plan: QueryPlan[_]): (Int, Map[String, String]) = { - var numGlutenNodes = 0 - val fallbackNodeToReason = new mutable.HashMap[String, String] + type FallbackInfo = (Int, Map[String, String]) - def addFallbackNodeWithReason(p: SparkPlan, reason: String): Unit = { - p.getTagValue(QueryPlan.OP_ID_TAG).foreach { - opId => - // e.g., 002 project, it is used to help analysis by `substring(4)` - val formattedNodeName = f"$opId%03d ${p.nodeName}" - fallbackNodeToReason.put(formattedNodeName, reason) - } + def addFallbackNodeWithReason( + p: SparkPlan, + reason: String, + fallbackNodeToReason: mutable.HashMap[String, String]): Unit = { + p.getTagValue(QueryPlan.OP_ID_TAG).foreach { + opId => + // e.g., 002 project, it is used to help analysis by `substring(4)` + val formattedNodeName = f"$opId%03d ${p.nodeName}" + fallbackNodeToReason.put(formattedNodeName, reason) } + } + + def handleVanillaSparkPlan( + p: SparkPlan, + fallbackNodeToReason: mutable.HashMap[String, String] + ): Unit = { + p.logicalLink.flatMap(_.getTagValue(FALLBACK_REASON_TAG)) match { + case Some(reason) => addFallbackNodeWithReason(p, reason, fallbackNodeToReason) + case _ => + // If the SparkPlan does not have fallback reason, then there are two options: + // 1. Gluten ignore that plan and it's a kind of fallback + // 2. Gluten does not support it without the fallback reason + addFallbackNodeWithReason( + p, + "Gluten does not touch it or does not support it", + fallbackNodeToReason) + } + } + + private def collectFallbackNodes(plan: QueryPlan[_]): FallbackInfo = { + var numGlutenNodes = 0 + val fallbackNodeToReason = new mutable.HashMap[String, String] def collect(tmp: QueryPlan[_]): Unit = { tmp.foreachUp { + case _: ExecutedCommandExec => + case _: CommandResultExec => + case _: V2CommandExec => + case _: DataWritingCommandExec => case _: WholeStageCodegenExec => case _: WholeStageTransformer => case _: InputAdapter => - case p: AdaptiveSparkPlanExec => collect(p.executedPlan) + case _: ColumnarToRowTransition => + case _: RowToColumnarTransition => case p: QueryStageExec => collect(p.plan) case p: GlutenPlan => numGlutenNodes += 1 @@ -65,17 +94,11 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { if (InMemoryTableScanHelper.isGlutenTableCache(i)) { numGlutenNodes += 1 } else { - addFallbackNodeWithReason(i, "Columnar table cache is disabled") + addFallbackNodeWithReason(i, "Columnar table cache is disabled", fallbackNodeToReason) } + case _: AQEShuffleReadExec => // Ignore case p: SparkPlan => - p.logicalLink.flatMap(_.getTagValue(FALLBACK_REASON_TAG)) match { - case Some(reason) => addFallbackNodeWithReason(p, reason) - case _ => - // If the SparkPlan does not have fallback reason, then there are two options: - // 1. Gluten ignore that plan and it's a kind of fallback - // 2. Gluten does not support it without the fallback reason - addFallbackNodeWithReason(p, "Gluten does not touch it or does not support it") - } + handleVanillaSparkPlan(p, fallbackNodeToReason) p.innerChildren.foreach(collect) case _ => } @@ -120,11 +143,12 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { */ def processPlan[T <: QueryPlan[T]]( plan: T, - append: String => Unit): (Int, Map[String, String]) = { + append: String => Unit, + collectFallbackFunc: Option[QueryPlan[_] => FallbackInfo] = None): FallbackInfo = { try { // Initialize a reference-unique set of Operators to avoid accdiental overwrites and to allow // intentional overwriting of IDs generated in previous AQE iteration - val operators = newSetFromMap[QueryPlan[_]](new IdentityHashMap()) + val operators = newSetFromMap[QueryPlan[_]](new util.IdentityHashMap()) // Initialize an array of ReusedExchanges to help find Adaptively Optimized Out // Exchanges as part of SPARK-42753 val reusedExchanges = ArrayBuffer.empty[ReusedExchangeExec] @@ -186,7 +210,11 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { append("\n") } - collectFallbackNodes(plan) + if (collectFallbackFunc.isEmpty) { + collectFallbackNodes(plan) + } else { + collectFallbackFunc.get.apply(plan) + } } finally { removeTags(plan) } @@ -224,7 +252,7 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { private def generateOperatorIDs( plan: QueryPlan[_], startOperatorID: Int, - visited: Set[QueryPlan[_]], + visited: util.Set[QueryPlan[_]], reusedExchanges: ArrayBuffer[ReusedExchangeExec], addReusedExchanges: Boolean): Int = { var currentOperationID = startOperatorID @@ -310,11 +338,11 @@ object GlutenExplainUtils extends AdaptiveSparkPlanHelper { * Returns the operator identifier for the supplied plan by retrieving the `operationId` tag * value. */ - def getOpId(plan: QueryPlan[_]): String = { + private def getOpId(plan: QueryPlan[_]): String = { plan.getTagValue(QueryPlan.OP_ID_TAG).map(v => s"$v").getOrElse("unknown") } - def removeTags(plan: QueryPlan[_]): Unit = { + private def removeTags(plan: QueryPlan[_]): Unit = { def remove(p: QueryPlan[_], children: Seq[QueryPlan[_]]): Unit = { p.unsetTagValue(QueryPlan.OP_ID_TAG) children.foreach(removeTags) diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala new file mode 100644 index 000000000000..f79266706bec --- /dev/null +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/GlutenImplicits.scala @@ -0,0 +1,212 @@ +/* + * 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 io.glutenproject.execution.WholeStageTransformer +import io.glutenproject.extension.{GlutenPlan, InMemoryTableScanHelper} + +import org.apache.spark.sql.{AnalysisException, Dataset} +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.GlutenExplainUtils._ +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AQEShuffleReadExec, 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.v2.V2CommandExec +import org.apache.spark.sql.internal.SQLConf + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +// spotless:off +/** + * A helper class to get the Gluten fallback summary from a Spark [[Dataset]]. + * + * Note that, if AQE is enabled, but the query is not materialized, then this method will re-plan + * the query execution with disabled AQE. It is a workaround to get the final plan, and it may + * cause the inconsistent results with a materialized query. However, we have no choice. + * + * For example: + * + * {{{ + * import org.apache.spark.sql.execution.GlutenImplicits._ + * val df = spark.sql("SELECT * FROM t") + * df.fallbackSummary + * }}} + */ +// spotless:on +object GlutenImplicits { + + case class FallbackSummary( + numGlutenNodes: Int, + numFallbackNodes: Int, + physicalPlanDescription: Seq[String], + fallbackNodeToReason: Seq[Map[String, String]]) {} + + private[sql] def withSQLConf[T](pairs: (String, String)*)(f: => T): T = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { + key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + keys.zip(values).foreach { + case (k, v) => + if (SQLConf.isStaticConfigKey(k)) { + throw new AnalysisException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f + finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + + implicit class DatasetTransformer[T](dateset: Dataset[T]) { + private def isFinalAdaptivePlan(p: AdaptiveSparkPlanExec): Boolean = { + val args = p.argString(Int.MaxValue) + val index = args.indexOf("isFinalPlan=") + assert(index >= 0) + args.substring(index + "isFinalPlan=".length).trim.toBoolean + } + + private def collectFallbackNodes(plan: QueryPlan[_]): FallbackInfo = { + var numGlutenNodes = 0 + val fallbackNodeToReason = new mutable.HashMap[String, String] + + def collect(tmp: QueryPlan[_]): Unit = { + tmp.foreachUp { + case _: ExecutedCommandExec => + case _: CommandResultExec => + case _: V2CommandExec => + case _: DataWritingCommandExec => + case _: WholeStageCodegenExec => + case _: WholeStageTransformer => + case _: InputAdapter => + case _: ColumnarToRowTransition => + case _: RowToColumnarTransition => + case p: AdaptiveSparkPlanExec if isFinalAdaptivePlan(p) => + collect(p.executedPlan) + case p: AdaptiveSparkPlanExec => + // if we are here that means we are inside table cache. + val (innerNumGlutenNodes, innerFallbackNodeToReason) = + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // re-plan manually to skip cached data + val newSparkPlan = QueryExecution.createSparkPlan( + dateset.sparkSession, + dateset.sparkSession.sessionState.planner, + p.inputPlan.logicalLink.get) + val newExecutedPlan = QueryExecution.prepareExecutedPlan( + dateset.sparkSession, + newSparkPlan + ) + processPlan( + newExecutedPlan, + new PlanStringConcat().append, + Some(plan => collectFallbackNodes(plan))) + } + numGlutenNodes += innerNumGlutenNodes + fallbackNodeToReason.++=(innerFallbackNodeToReason) + case p: QueryStageExec => collect(p.plan) + case p: GlutenPlan => + numGlutenNodes += 1 + p.innerChildren.foreach(collect) + case i: InMemoryTableScanExec => + if (InMemoryTableScanHelper.isGlutenTableCache(i)) { + numGlutenNodes += 1 + } else { + addFallbackNodeWithReason(i, "Columnar table cache is disabled", fallbackNodeToReason) + } + collect(i.relation.cachedPlan) + case _: AQEShuffleReadExec => // Ignore + case p: SparkPlan => + handleVanillaSparkPlan(p, fallbackNodeToReason) + p.innerChildren.foreach(collect) + case _ => + } + } + + collect(plan) + (numGlutenNodes, fallbackNodeToReason.toMap) + } + + private def collectQueryExecutionFallbackSummary(qe: QueryExecution): FallbackSummary = { + var totalNumGlutenNodes = 0 + var totalNumFallbackNodes = 0 + val totalPhysicalPlanDescription = new ArrayBuffer[String]() + val totalFallbackNodeToReason = new ArrayBuffer[Map[String, String]]() + + def handlePlanWithAQEAndTableCache( + plan: SparkPlan, + logicalPlan: LogicalPlan, + isMaterialized: Boolean): Unit = { + val concat = new PlanStringConcat() + val collectFallbackFunc = Some(plan => collectFallbackNodes(plan)) + val (numGlutenNodes, fallbackNodeToReason) = if (!isMaterialized) { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + // AQE is not materialized, so the columnar rules are not applied. + // For this case, We apply columnar rules manually with disable AQE. + val qe = dateset.sparkSession.sessionState.executePlan(logicalPlan) + processPlan(qe.executedPlan, concat.append, collectFallbackFunc) + } + } else { + processPlan(plan, concat.append, collectFallbackFunc) + } + totalNumGlutenNodes += numGlutenNodes + totalNumFallbackNodes += fallbackNodeToReason.size + totalPhysicalPlanDescription.append(concat.toString()) + totalFallbackNodeToReason.append(fallbackNodeToReason) + } + + // For command-like query, e.g., `INSERT INTO TABLE ...` + qe.commandExecuted.foreach { + case r: CommandResult => + handlePlanWithAQEAndTableCache(r.commandPhysicalPlan, r.commandLogicalPlan, true) + case _ => // ignore + } + + // For query, e.g., `SELECT * FROM ...` + if (qe.executedPlan.find(_.isInstanceOf[CommandResultExec]).isEmpty) { + val isMaterialized = qe.executedPlan.find { + case a: AdaptiveSparkPlanExec if isFinalAdaptivePlan(a) => true + case _ => false + }.isDefined + handlePlanWithAQEAndTableCache(qe.executedPlan, qe.analyzed, isMaterialized) + } + + FallbackSummary( + totalNumGlutenNodes, + totalNumFallbackNodes, + totalPhysicalPlanDescription, + totalFallbackNodeToReason + ) + } + + def fallbackSummary(): FallbackSummary = { + collectQueryExecutionFallbackSummary(dateset.queryExecution) + } + } +} diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/adaptive/ColumnarAQEShuffleReadExec.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/adaptive/ColumnarAQEShuffleReadExec.scala deleted file mode 100644 index f208a5cf8ecb..000000000000 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/adaptive/ColumnarAQEShuffleReadExec.scala +++ /dev/null @@ -1,269 +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.adaptive - -import io.glutenproject.extension.GlutenPlan - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning, SinglePartition, UnknownPartitioning} -import org.apache.spark.sql.catalyst.trees.CurrentOrigin -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} -import org.apache.spark.sql.vectorized.ColumnarBatch - -import scala.collection.mutable.ArrayBuffer - -/** - * A wrapper of shuffle query stage, which follows the given partition arrangement. - * - * @param child - * It is usually `ShuffleQueryStageExec`, but can be the shuffle exchange node during - * canonicalization. - * @param partitionSpecs - * The partition specs that defines the arrangement. - */ -case class ColumnarAQEShuffleReadExec(child: SparkPlan, partitionSpecs: Seq[ShufflePartitionSpec]) - extends UnaryExecNode - with GlutenPlan { - // We don't extends AQEShuffleReadExec since it has private constructor - - override lazy val outputPartitioning: Partitioning = { - // If it is a local shuffle reader with one mapper per task, then the output partitioning is - // the same as the plan before shuffle. - // TODO this check is based on assumptions of callers' behavior but is sufficient for now. - if ( - partitionSpecs.forall(_.isInstanceOf[PartialMapperPartitionSpec]) && - partitionSpecs.map(_.asInstanceOf[PartialMapperPartitionSpec].mapIndex).toSet.size == - partitionSpecs.length - ) { - child match { - case ShuffleQueryStageExec(_, s: ColumnarShuffleExchangeExec, _) => - s.child.outputPartitioning - case ShuffleQueryStageExec( - _, - r @ ReusedExchangeExec(_, s: ColumnarShuffleExchangeExec), - _) => - s.child.outputPartitioning match { - case e: Expression => r.updateAttr(e).asInstanceOf[Partitioning] - case other => other - } - case _ => - throw new IllegalStateException("operating on canonicalization plan") - } - } else if (isCoalescedRead) { - // For coalesced shuffle read, the data distribution is not changed, only the number of - // partitions is changed. - child.outputPartitioning match { - case h: HashPartitioning => - CurrentOrigin.withOrigin(h.origin)(h.copy(numPartitions = partitionSpecs.length)) - case r: RangePartitioning => - CurrentOrigin.withOrigin(r.origin)(r.copy(numPartitions = partitionSpecs.length)) - // This can only happen for `REBALANCE_PARTITIONS_BY_NONE`, which uses - // `RoundRobinPartitioning` but we don't need to retain the number of partitions. - case r: RoundRobinPartitioning => - r.copy(numPartitions = partitionSpecs.length) - case other @ SinglePartition => - throw new IllegalStateException( - "Unexpected partitioning for coalesced shuffle read: " + other) - case _ => - // Spark plugins may have custom partitioning and may replace this operator - // during the postStageOptimization phase, so return UnknownPartitioning here - // rather than throw an exception - UnknownPartitioning(partitionSpecs.length) - } - } else { - UnknownPartitioning(partitionSpecs.length) - } - } - private var cachedShuffleRDD: RDD[ColumnarBatch] = null - - override def output: Seq[Attribute] = child.output - - override def supportsColumnar: Boolean = true - - override def doExecuteColumnar(): RDD[ColumnarBatch] = { - if (cachedShuffleRDD == null) { - cachedShuffleRDD = child match { - case stage: ShuffleQueryStageExec => - sendDriverMetrics() - new ShuffledColumnarBatchRDD( - stage.shuffle - .asInstanceOf[ColumnarShuffleExchangeExec] - .columnarShuffleDependency, - stage.shuffle.asInstanceOf[ColumnarShuffleExchangeExec].readMetrics, - partitionSpecs.toArray - ) - case _ => - throw new IllegalStateException("operating on canonicalized plan") - } - } - cachedShuffleRDD - } - - override protected def doExecute(): RDD[InternalRow] = - throw new UnsupportedOperationException() - - override def stringArgs: Iterator[Any] = { - val desc = if (isLocalRead) { - "local" - } else if (hasCoalescedPartition && hasSkewedPartition) { - "coalesced and skewed" - } else if (hasCoalescedPartition) { - "coalesced" - } else if (hasSkewedPartition) { - "skewed" - } else { - "" - } - Iterator(desc) - } - - /** Returns true iff some partitions were actually combined */ - private def isCoalescedSpec(spec: ShufflePartitionSpec) = spec match { - case CoalescedPartitionSpec(0, 0, _) => true - case s: CoalescedPartitionSpec => s.endReducerIndex - s.startReducerIndex > 1 - case _ => false - } - - /** Returns true iff some non-empty partitions were combined */ - def hasCoalescedPartition: Boolean = { - partitionSpecs.exists(isCoalescedSpec) - } - - def hasSkewedPartition: Boolean = - partitionSpecs.exists(_.isInstanceOf[PartialReducerPartitionSpec]) - - def isLocalRead: Boolean = - partitionSpecs.exists(_.isInstanceOf[PartialMapperPartitionSpec]) || - partitionSpecs.exists(_.isInstanceOf[CoalescedMapperPartitionSpec]) - - def isCoalescedRead: Boolean = { - partitionSpecs.sliding(2).forall { - // A single partition spec which is `CoalescedPartitionSpec` also means coalesced read. - case Seq(_: CoalescedPartitionSpec) => true - case Seq(l: CoalescedPartitionSpec, r: CoalescedPartitionSpec) => - l.endReducerIndex <= r.startReducerIndex - case _ => false - } - } - - private def shuffleStage = child match { - case stage: ShuffleQueryStageExec => Some(stage) - case _ => None - } - - @transient private lazy val partitionDataSizes: Option[Seq[Long]] = { - if (!isLocalRead && shuffleStage.get.mapStats.isDefined) { - Some(partitionSpecs.map { - case p: CoalescedPartitionSpec => - assert(p.dataSize.isDefined) - p.dataSize.get - case p: PartialReducerPartitionSpec => p.dataSize - case p => throw new IllegalStateException(s"unexpected $p") - }) - } else { - None - } - } - - private def sendDriverMetrics(): Unit = { - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - val driverAccumUpdates = ArrayBuffer.empty[(Long, Long)] - - val numPartitionsMetric = metrics("numPartitions") - numPartitionsMetric.set(partitionSpecs.length) - driverAccumUpdates += (numPartitionsMetric.id -> partitionSpecs.length.toLong) - - if (hasSkewedPartition) { - val skewedSpecs = partitionSpecs.collect { case p: PartialReducerPartitionSpec => p } - - val skewedPartitions = metrics("numSkewedPartitions") - val skewedSplits = metrics("numSkewedSplits") - - val numSkewedPartitions = skewedSpecs.map(_.reducerIndex).distinct.length - val numSplits = skewedSpecs.length - - skewedPartitions.set(numSkewedPartitions) - driverAccumUpdates += (skewedPartitions.id -> numSkewedPartitions) - - skewedSplits.set(numSplits) - driverAccumUpdates += (skewedSplits.id -> numSplits) - } - - if (hasCoalescedPartition) { - val numCoalescedPartitionsMetric = metrics("numCoalescedPartitions") - val x = partitionSpecs.count(isCoalescedSpec) - numCoalescedPartitionsMetric.set(x) - driverAccumUpdates += numCoalescedPartitionsMetric.id -> x - } - - partitionDataSizes.foreach { - dataSizes => - val partitionDataSizeMetrics = metrics("partitionDataSize") - driverAccumUpdates ++= dataSizes.map(partitionDataSizeMetrics.id -> _) - // Set sum value to "partitionDataSize" metric. - partitionDataSizeMetrics.set(dataSizes.sum) - } - - SQLMetrics.postDriverMetricsUpdatedByValue(sparkContext, executionId, driverAccumUpdates.toSeq) - } - - // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. - @transient override lazy val metrics: Map[String, SQLMetric] = { - if (shuffleStage.isDefined) { - Map("numPartitions" -> SQLMetrics.createMetric(sparkContext, "number of partitions")) ++ { - if (isLocalRead) { - // We split the mapper partition evenly when creating local shuffle read, so no - // data size info is available. - Map.empty - } else { - Map( - "partitionDataSize" -> - SQLMetrics.createSizeMetric(sparkContext, "partition data size")) - } - } ++ { - if (hasSkewedPartition) { - Map( - "numSkewedPartitions" -> - SQLMetrics.createMetric(sparkContext, "number of skewed partitions"), - "numSkewedSplits" -> - SQLMetrics.createMetric(sparkContext, "number of skewed partition splits") - ) - } else { - Map.empty - } - } ++ { - if (hasCoalescedPartition) { - Map( - "numCoalescedPartitions" -> - SQLMetrics.createMetric(sparkContext, "number of coalesced partitions")) - } else { - Map.empty - } - } - } else { - // It's a canonicalized plan, no need to report metrics. - Map.empty - } - } - - override protected def withNewChildInternal(newChild: SparkPlan): ColumnarAQEShuffleReadExec = - copy(child = newChild) -} diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala b/gluten-core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala index c4df874e725d..4146cbc46b66 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExecTransformer.scala @@ -17,8 +17,7 @@ package org.apache.spark.api.python import io.glutenproject.backendsapi.BackendsApiManager -import io.glutenproject.execution.TransformContext -import io.glutenproject.execution.TransformSupport +import io.glutenproject.execution.{TransformContext, TransformSupport, UnaryTransformSupport} import io.glutenproject.expression._ import io.glutenproject.extension.ValidationResult import io.glutenproject.metrics.MetricsUpdater @@ -29,24 +28,22 @@ import io.glutenproject.substrait.extensions.ExtensionBuilder import io.glutenproject.substrait.rel._ import org.apache.spark.TaskContext -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.execution.SparkPlan import org.apache.spark.sql.execution.python.EvalPythonExec import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.vectorized.ColumnarBatch import com.google.protobuf.Any -import java.util.ArrayList +import java.util.{ArrayList => JArrayList, List => JList} case class EvalPythonExecTransformer( udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan) extends EvalPythonExec - with TransformSupport { + with UnaryTransformSupport { override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genFilterTransformerMetricsUpdater(metrics) @@ -60,37 +57,9 @@ case class EvalPythonExecTransformer( throw new IllegalStateException("EvalPythonExecTransformer doesn't support evaluate") } - override protected def doExecute(): RDD[InternalRow] = { - throw new UnsupportedOperationException(s"EvalPythonExecTransformer doesn't support doExecute") - } - override protected def withNewChildInternal(newChild: SparkPlan): EvalPythonExecTransformer = copy(udfs, resultAttrs, newChild) - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = child match { - case c: TransformSupport => - c.columnarInputRDDs - case _ => - Seq(child.executeColumnar()) - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = child match { - case c: TransformSupport => - val childPlans = c.getBuildPlans - childPlans :+ (this, null) - case _ => - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = child match { - case c: TransformSupport => - c.getStreamedLeafPlan - case _ => - this - } - - override def supportsColumnar: Boolean = true - override protected def doValidateInternal(): ValidationResult = { // All udfs should be scalar python udf for (udf <- udfs) { @@ -103,7 +72,7 @@ case class EvalPythonExecTransformer( val args = context.registeredFunction val operatorId = context.nextOperatorId(this.nodeName) - val expressionNodes = new java.util.ArrayList[ExpressionNode] + val expressionNodes = new JArrayList[ExpressionNode] child.output.zipWithIndex.foreach( x => expressionNodes.add(ExpressionBuilder.makeSelection(x._2))) udfs.foreach( @@ -125,7 +94,7 @@ case class EvalPythonExecTransformer( val args = context.registeredFunction val operatorId = context.nextOperatorId(this.nodeName) - val expressionNodes = new java.util.ArrayList[ExpressionNode] + val expressionNodes = new JArrayList[ExpressionNode] child.output.zipWithIndex.foreach( x => expressionNodes.add(ExpressionBuilder.makeSelection(x._2))) udfs.foreach( @@ -137,7 +106,7 @@ case class EvalPythonExecTransformer( val relNode = if (childCtx != null) { getRelNode(childCtx.root, expressionNodes, context, operatorId, child.output, false) } else { - val attrList = new java.util.ArrayList[Attribute]() + val attrList = new JArrayList[Attribute]() for (attr <- child.output) { attrList.add(attr) } @@ -150,7 +119,7 @@ case class EvalPythonExecTransformer( def getRelNode( input: RelNode, - expressionNodes: ArrayList[ExpressionNode], + expressionNodes: JList[ExpressionNode], context: SubstraitContext, operatorId: Long, inputAttributes: Seq[Attribute], @@ -159,7 +128,7 @@ case class EvalPythonExecTransformer( RelBuilder.makeProjectRel(input, expressionNodes, context, operatorId) } else { // Use a extension node to send the input types through Substrait plan for validation. - val inputTypeNodeList = new java.util.ArrayList[TypeNode]() + val inputTypeNodeList = new JArrayList[TypeNode]() for (attr <- inputAttributes) { inputTypeNodeList.add(ConverterUtils.getTypeNode(attr.dataType, attr.nullable)) } diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/hive/HivePartitionConverter.scala b/gluten-core/src/main/scala/org/apache/spark/sql/hive/HivePartitionConverter.scala index 7793c6ea7048..f441bd519323 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/hive/HivePartitionConverter.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/hive/HivePartitionConverter.scala @@ -28,9 +28,10 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.DataType import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.io.compress.CompressionCodecFactory +import org.apache.hadoop.mapred.FileInputFormat import java.net.URI @@ -46,6 +47,9 @@ class HivePartitionConverter(hadoopConf: Configuration, session: SparkSession) lazy val codecFactory: CompressionCodecFactory = new CompressionCodecFactory(hadoopConf) + + lazy val recursive: Boolean = hadoopConf.getBoolean(FileInputFormat.INPUT_DIR_RECURSIVE, false) + private def canBeSplit(filePath: Path): Boolean = { // Checks if file at path `filePath` can be split. // Uncompressed Hive Text files may be split. GZIP compressed files are not. @@ -92,11 +96,39 @@ class HivePartitionConverter(hadoopConf: Configuration, session: SparkSession) case (directory, partValues) => val path = new Path(directory) val fs = path.getFileSystem(hadoopConf) - val dirContents = fs.listStatus(path).filter(isNonEmptyDataFile) + val dirContents = fs + .listStatus(path) + .flatMap( + f => { + if (f.isFile) { + Seq(f) + } else if (recursive) { + addInputPathRecursively(fs, f) + } else { + Seq() + } + }) + .filter(isNonEmptyDataFile) PartitionDirectory(partValues, dirContents) } } + private def addInputPathRecursively(fs: FileSystem, files: FileStatus): Seq[FileStatus] = { + if (files.isFile) { + Seq(files) + } else { + fs.listStatus(files.getPath) + .flatMap( + file => { + if (file.isFile) { + Seq(file) + } else { + addInputPathRecursively(fs, file) + } + }) + } + } + private def createFilePartition( selectedPartitions: Seq[PartitionDirectory]): Seq[FilePartition] = { val maxSplitBytes = FilePartition.maxSplitBytes(session, selectedPartitions) diff --git a/gluten-core/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala b/gluten-core/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala index c1a27fe95466..bf8af495af97 100644 --- a/gluten-core/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala +++ b/gluten-core/src/main/scala/org/apache/spark/sql/hive/HiveTableScanExecTransformer.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.sql.hive -import io.glutenproject.GlutenConfig import io.glutenproject.backendsapi.BackendsApiManager import io.glutenproject.execution.{BasicScanExecTransformer, TransformContext} import io.glutenproject.extension.ValidationResult @@ -33,12 +32,15 @@ import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.hive.HiveTableScanExecTransformer._ +import org.apache.spark.sql.hive.client.HiveClientImpl import org.apache.spark.sql.hive.execution.HiveTableScanExec import org.apache.spark.sql.types.StructType import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat +import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.mapred.TextInputFormat import java.net.URI @@ -55,6 +57,13 @@ class HiveTableScanExecTransformer( @transient override lazy val metrics: Map[String, SQLMetric] = BackendsApiManager.getMetricsApiInstance.genHiveTableScanTransformerMetrics(sparkContext) + @transient private lazy val hiveQlTable = HiveClientImpl.toHiveTable(relation.tableMeta) + + @transient private lazy val tableDesc = new TableDesc( + hiveQlTable.getInputFormatClass, + hiveQlTable.getOutputFormatClass, + hiveQlTable.getMetadata) + override def filterExprs(): Seq[Expression] = Seq.empty override def outputAttributes(): Seq[Attribute] = output @@ -70,23 +79,9 @@ class HiveTableScanExecTransformer( Seq.empty } - override def columnarInputRDDs: Seq[RDD[ColumnarBatch]] = { - Seq.empty - } - - override def getBuildPlans: Seq[(SparkPlan, SparkPlan)] = { - Seq((this, null)) - } - - override def getStreamedLeafPlan: SparkPlan = { - this - } - override def metricsUpdater(): MetricsUpdater = BackendsApiManager.getMetricsApiInstance.genHiveTableScanTransformerMetricsUpdater(metrics) - override def supportsColumnar(): Boolean = GlutenConfig.getConf.enableColumnarIterator - override def doExecuteColumnar(): RDD[ColumnarBatch] = { doExecuteColumnarInternal() } @@ -119,6 +114,9 @@ class HiveTableScanExecTransformer( case Some(inputFormat) if ORC_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => ReadFileFormat.OrcReadFormat + case Some(inputFormat) + if PARQUET_INPUT_FORMAT_CLASS.isAssignableFrom(Utils.classForName(inputFormat)) => + ReadFileFormat.ParquetReadFormat case _ => ReadFileFormat.UnknownFormat } } @@ -133,6 +131,7 @@ class HiveTableScanExecTransformer( case _ => options += ("field_delimiter" -> DEFAULT_FIELD_DELIMITER.toString) options += ("nullValue" -> NULL_VALUE.toString) + options += ("escape" -> "\\") } options @@ -144,7 +143,11 @@ class HiveTableScanExecTransformer( transformCtx.root != null && transformCtx.root.isInstanceOf[ReadRelNode] ) { - val properties = relation.tableMeta.storage.properties ++ relation.tableMeta.properties + var properties: Map[String, String] = Map() + tableDesc.getProperties + .entrySet() + .forEach(e => properties += (e.getKey.toString -> e.getValue.toString)) + var options: Map[String, String] = createDefaultTextOption() // property key string read from org.apache.hadoop.hive.serde.serdeConstants properties.foreach { @@ -204,7 +207,8 @@ object HiveTableScanExecTransformer { Utils.classForName("org.apache.hadoop.mapred.TextInputFormat") val ORC_INPUT_FORMAT_CLASS: Class[OrcInputFormat] = Utils.classForName("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat") - + val PARQUET_INPUT_FORMAT_CLASS: Class[MapredParquetInputFormat] = + Utils.classForName("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat") def isHiveTableScan(plan: SparkPlan): Boolean = { plan.isInstanceOf[HiveTableScanExec] } diff --git a/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala b/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala index e57badad5faa..4cb33186ad1e 100644 --- a/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala +++ b/gluten-core/src/main/scala/org/apache/spark/util/TaskResources.scala @@ -36,6 +36,20 @@ object TaskResources extends TaskListener with Logging { } val ACCUMULATED_LEAK_BYTES = new AtomicLong(0L) + // For testing purpose only + private var fallbackRegistry: Option[TaskResourceRegistry] = None + + // For testing purpose only + def setFallbackRegistry(r: TaskResourceRegistry): Unit = { + fallbackRegistry = Some(r) + } + + // For testing purpose only + def unsetFallbackRegistry(): Unit = { + fallbackRegistry.foreach(r => r.releaseAll()) + fallbackRegistry = None + } + private val RESOURCE_REGISTRIES = new java.util.IdentityHashMap[TaskContext, TaskResourceRegistry]() @@ -52,7 +66,11 @@ object TaskResources extends TaskListener with Logging { logWarning( "Using the fallback instance of TaskResourceRegistry. " + "This should only happen when call is not from Spark task.") - throw new IllegalStateException("Found a caller not in Spark task scope.") + return fallbackRegistry match { + case Some(r) => r + case _ => + throw new IllegalStateException("No fallback instance of TaskResourceRegistry found.") + } } val tc = getLocalTaskContext() RESOURCE_REGISTRIES.synchronized { diff --git a/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala b/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala index ae0ae9edcdc8..621fa63ef62a 100644 --- a/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala +++ b/gluten-core/src/test/scala/io/glutenproject/execution/WholeStageTransformerSuite.scala @@ -39,6 +39,9 @@ abstract class WholeStageTransformerSuite extends GlutenQueryTest with SharedSpa protected val fileFormat: String protected val logLevel: String = "WARN" + protected val TPCHTableNames: Seq[String] = + Seq("customer", "lineitem", "nation", "orders", "part", "partsupp", "region", "supplier") + protected var TPCHTables: Map[String, DataFrame] = _ private val isFallbackCheckDisabled0 = new AtomicBoolean(false) @@ -61,15 +64,7 @@ abstract class WholeStageTransformerSuite extends GlutenQueryTest with SharedSpa } protected def createTPCHNotNullTables(): Unit = { - TPCHTables = Seq( - "customer", - "lineitem", - "nation", - "orders", - "part", - "partsupp", - "region", - "supplier").map { + TPCHTables = TPCHTableNames.map { table => val tableDir = getClass.getResource(resourcePath).getFile val tablePath = new File(tableDir, table).getAbsolutePath diff --git a/gluten-core/src/test/scala/io/glutenproject/utils/IteratorSuite.scala b/gluten-core/src/test/scala/io/glutenproject/utils/IteratorSuite.scala new file mode 100644 index 000000000000..88b2eb63126b --- /dev/null +++ b/gluten-core/src/test/scala/io/glutenproject/utils/IteratorSuite.scala @@ -0,0 +1,144 @@ +/* + * 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 io.glutenproject.utils + +import org.apache.spark.util.{TaskResourceRegistry, TaskResources} + +import org.scalatest.funsuite.AnyFunSuite + +class IteratorSuite extends AnyFunSuite { + test("Trivial wrapping") { + val strings = Array[String]("one", "two", "three") + val itr = strings.toIterator + val wrapped = Iterators + .wrap(itr) + .create() + assertResult(strings) { + wrapped.toArray + } + } + + test("Complete iterator") { + var completeCount = 0 + withFakeTaskContext { + val strings = Array[String]("one", "two", "three") + val itr = strings.toIterator + val wrapped = Iterators + .wrap(itr) + .recycleIterator { + completeCount += 1 + } + .create() + assertResult(strings) { + wrapped.toArray + } + assert(completeCount == 1) + } + assert(completeCount == 1) + } + + test("Complete intermediate iterator") { + var completeCount = 0 + withFakeTaskContext { + val strings = Array[String]("one", "two", "three") + val itr = strings.toIterator + val _ = Iterators + .wrap(itr) + .recycleIterator { + completeCount += 1 + } + .create() + assert(completeCount == 0) + } + assert(completeCount == 1) + } + + test("Close payload") { + var closeCount = 0 + withFakeTaskContext { + val strings = Array[String]("one", "two", "three") + val itr = strings.toIterator + val wrapped = Iterators + .wrap(itr) + .recyclePayload { _: String => closeCount += 1 } + .create() + assertResult(strings) { + wrapped.toArray + } + assert(closeCount == 3) + } + assert(closeCount == 3) + } + + test("Close intermediate payload") { + var closeCount = 0 + withFakeTaskContext { + val strings = Array[String]("one", "two", "three") + val itr = strings.toIterator + val wrapped = Iterators + .wrap(itr) + .recyclePayload { _: String => closeCount += 1 } + .create() + assertResult(strings.take(2)) { + wrapped.take(2).toArray + } + assert(closeCount == 1) // the first one is closed after consumed + } + assert(closeCount == 2) // the second one is closed on task exit + } + + test("Protect invocation flow") { + var hasNextCallCount = 0 + var nextCallCount = 0 + val itr = new Iterator[Any] { + override def hasNext: Boolean = { + hasNextCallCount += 1 + true + } + + override def next(): Any = { + nextCallCount += 1 + new Object + } + } + val wrapped = Iterators + .wrap(itr) + .protectInvocationFlow() + .create() + wrapped.hasNext + assert(hasNextCallCount == 1) + assert(nextCallCount == 0) + wrapped.hasNext + assert(hasNextCallCount == 1) + assert(nextCallCount == 0) + wrapped.next + assert(hasNextCallCount == 1) + assert(nextCallCount == 1) + wrapped.next + assert(hasNextCallCount == 2) + assert(nextCallCount == 2) + } + + private def withFakeTaskContext(body: => Unit): Unit = { + TaskResources.setFallbackRegistry(new TaskResourceRegistry) + try { + body + } finally { + TaskResources.unsetFallbackRegistry() + } + } +} diff --git a/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala b/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala index c1e74b6a20a4..b1f772e5cdff 100644 --- a/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala +++ b/gluten-core/src/test/scala/org/apache/spark/softaffinity/SoftAffinitySuite.scala @@ -20,6 +20,7 @@ import io.glutenproject.GlutenConfig import io.glutenproject.execution.{GlutenMergeTreePartition, GlutenPartition} import io.glutenproject.softaffinity.SoftAffinityManager import io.glutenproject.softaffinity.scheduler.SoftAffinityListener +import io.glutenproject.sql.shims.SparkShimLoader import io.glutenproject.substrait.plan.PlanBuilder import org.apache.spark.SparkConf @@ -28,7 +29,7 @@ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.PredicateHelper -import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} +import org.apache.spark.sql.execution.datasources.FilePartition import org.apache.spark.sql.test.SharedSparkSession class SoftAffinitySuite extends QueryTest with SharedSparkSession with PredicateHelper { @@ -42,14 +43,28 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val partition = FilePartition( 0, Seq( - PartitionedFile(InternalRow.empty, "fakePath0", 0, 100, Array("host-1", "host-2")), - PartitionedFile(InternalRow.empty, "fakePath1", 0, 200, Array("host-2", "host-3")) + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath0", + 0, + 100, + Array("host-1", "host-2") + ), + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath1", + 0, + 200, + Array("host-2", "host-3") + ) ).toArray ) - val locations = SoftAffinityUtil.getFilePartitionLocations(partition) + val locations = SoftAffinityUtil.getFilePartitionLocations( + partition.files.map(_.filePath.toString), + partition.preferredLocations()) - val nativePartition = new GlutenPartition(0, PlanBuilder.empty().toProtobuf, locations) + val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations) assertResult(Set("host-1", "host-2", "host-3")) { nativePartition.preferredLocations().toSet } @@ -59,14 +74,28 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val partition = FilePartition( 0, Seq( - PartitionedFile(InternalRow.empty, "fakePath0", 0, 100, Array("host-1", "host-2")), - PartitionedFile(InternalRow.empty, "fakePath1", 0, 200, Array("host-4", "host-5")) + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath0", + 0, + 100, + Array("host-1", "host-2") + ), + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath1", + 0, + 200, + Array("host-4", "host-5") + ) ).toArray ) - val locations = SoftAffinityUtil.getFilePartitionLocations(partition) + val locations = SoftAffinityUtil.getFilePartitionLocations( + partition.files.map(_.filePath.toString), + partition.preferredLocations()) - val nativePartition = new GlutenPartition(0, PlanBuilder.empty().toProtobuf, locations) + val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations) assertResult(Set("host-1", "host-4", "host-5")) { nativePartition.preferredLocations().toSet @@ -77,14 +106,28 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val partition = FilePartition( 0, Seq( - PartitionedFile(InternalRow.empty, "fakePath0", 0, 100, Array("host-1", "host-2")), - PartitionedFile(InternalRow.empty, "fakePath1", 0, 200, Array("host-5", "host-6")) + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath0", + 0, + 100, + Array("host-1", "host-2") + ), + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath1", + 0, + 200, + Array("host-5", "host-6") + ) ).toArray ) - val locations = SoftAffinityUtil.getFilePartitionLocations(partition) + val locations = SoftAffinityUtil.getFilePartitionLocations( + partition.files.map(_.filePath.toString), + partition.preferredLocations()) - val nativePartition = new GlutenPartition(0, PlanBuilder.empty().toProtobuf, locations) + val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations) assertResult(Set("executor_host-2_2", "executor_host-1_0")) { nativePartition.preferredLocations().toSet @@ -96,7 +139,7 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val locations = SoftAffinityUtil.getNativeMergeTreePartitionLocations(partition) - val nativePartition = new GlutenPartition(0, PlanBuilder.empty().toProtobuf, locations) + val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations) assertResult(Set("executor_host-1_1")) { nativePartition.preferredLocations().toSet @@ -107,14 +150,28 @@ class SoftAffinitySuite extends QueryTest with SharedSparkSession with Predicate val partition = FilePartition( 0, Seq( - PartitionedFile(InternalRow.empty, "fakePath0", 0, 100, Array("host-1", "host-2")), - PartitionedFile(InternalRow.empty, "fakePath1", 0, 200, Array("host-5", "host-6")) + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath0", + 0, + 100, + Array("host-1", "host-2") + ), + SparkShimLoader.getSparkShims.generatePartitionedFile( + InternalRow.empty, + "fakePath1", + 0, + 200, + Array("host-5", "host-6") + ) ).toArray ) - val locations = SoftAffinityUtil.getFilePartitionLocations(partition) + val locations = SoftAffinityUtil.getFilePartitionLocations( + partition.files.map(_.filePath.toString), + partition.preferredLocations()) - val nativePartition = new GlutenPartition(0, PlanBuilder.empty().toProtobuf, locations) + val nativePartition = GlutenPartition(0, PlanBuilder.EMPTY_PLAN, locations) assertResult(Set("host-1", "host-5", "host-6")) { nativePartition.preferredLocations().toSet diff --git a/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatchJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatchJniWrapper.java index eaf5adc21a8c..08468df15c56 100644 --- a/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatchJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatchJniWrapper.java @@ -16,23 +16,23 @@ */ package io.glutenproject.columnarbatch; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; -public class ColumnarBatchJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class ColumnarBatchJniWrapper implements RuntimeAware { + private final Runtime runtime; - private ColumnarBatchJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private ColumnarBatchJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static ColumnarBatchJniWrapper create() { - return new ColumnarBatchJniWrapper(ExecutionCtxs.contextInstance()); + return new ColumnarBatchJniWrapper(Runtimes.contextInstance()); } - public static ColumnarBatchJniWrapper forCtx(ExecutionCtx ctx) { - return new ColumnarBatchJniWrapper(ctx); + public static ColumnarBatchJniWrapper forRuntime(Runtime runtime) { + return new ColumnarBatchJniWrapper(runtime); } public native long createWithArrowArray(long cSchema, long cArray); @@ -59,7 +59,7 @@ public native long select( public native void close(long batch); @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } } diff --git a/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatches.java b/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatches.java index b2fc095242d4..25f3c16a89ce 100644 --- a/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatches.java +++ b/gluten-data/src/main/java/io/glutenproject/columnarbatch/ColumnarBatches.java @@ -17,8 +17,8 @@ package io.glutenproject.columnarbatch; import io.glutenproject.exception.GlutenException; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.Runtimes; import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators; import io.glutenproject.memory.nmm.NativeMemoryManager; import io.glutenproject.utils.ArrowAbiUtil; @@ -131,7 +131,7 @@ public static ColumnarBatch select( long outputBatchHandle = ColumnarBatchJniWrapper.create() .select(nmm.getNativeInstanceHandle(), iv.handle(), columnIndices); - return create(iv.ctx(), outputBatchHandle); + return create(iv.runtime(), outputBatchHandle); case HEAVY: return new ColumnarBatch( Arrays.stream(columnIndices).mapToObj(batch::column).toArray(ColumnVector[]::new), @@ -179,7 +179,7 @@ private static ColumnarBatch load(BufferAllocator allocator, ColumnarBatch input ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema arrowSchema = ArrowSchema.allocateNew(allocator); CDataDictionaryProvider provider = new CDataDictionaryProvider()) { - ColumnarBatchJniWrapper.forCtx(iv.ctx()) + ColumnarBatchJniWrapper.forRuntime(iv.runtime()) .exportToArrow(iv.handle(), cSchema.memoryAddress(), cArray.memoryAddress()); Data.exportSchema( @@ -215,15 +215,15 @@ private static ColumnarBatch offload(BufferAllocator allocator, ColumnarBatch in if (input.numCols() == 0) { throw new IllegalArgumentException("batch with zero columns cannot be offloaded"); } - final ExecutionCtx ctx = ExecutionCtxs.contextInstance(); + final Runtime runtime = Runtimes.contextInstance(); try (ArrowArray cArray = ArrowArray.allocateNew(allocator); ArrowSchema cSchema = ArrowSchema.allocateNew(allocator)) { ArrowAbiUtil.exportFromSparkColumnarBatch( ArrowBufferAllocators.contextInstance(), input, cSchema, cArray); long handle = - ColumnarBatchJniWrapper.forCtx(ctx) + ColumnarBatchJniWrapper.forRuntime(runtime) .createWithArrowArray(cSchema.memoryAddress(), cArray.memoryAddress()); - ColumnarBatch output = ColumnarBatches.create(ctx, handle); + ColumnarBatch output = ColumnarBatches.create(runtime, handle); // Follow input's reference count. This might be optimized using // automatic clean-up or once the extensibility of ColumnarBatch is enriched @@ -332,18 +332,18 @@ public static long compose(ColumnarBatch... batches) { Arrays.stream(batches) .map(ColumnarBatches::getIndicatorVector) .toArray(IndicatorVector[]::new); - // We assume all input batches should be managed by same ExecutionCtx. + // We assume all input batches should be managed by same Runtime. // FIXME: The check could be removed to adopt ownership-transfer semantic - final ExecutionCtx[] ctxs = - Arrays.stream(ivs).map(IndicatorVector::ctx).distinct().toArray(ExecutionCtx[]::new); + final Runtime[] ctxs = + Arrays.stream(ivs).map(IndicatorVector::runtime).distinct().toArray(Runtime[]::new); Preconditions.checkState( - ctxs.length == 1, "All input batches should be managed by same ExecutionCtx."); + ctxs.length == 1, "All input batches should be managed by same Runtime."); final long[] handles = Arrays.stream(ivs).mapToLong(IndicatorVector::handle).toArray(); - return ColumnarBatchJniWrapper.forCtx(ctxs[0]).compose(handles); + return ColumnarBatchJniWrapper.forRuntime(ctxs[0]).compose(handles); } - public static ColumnarBatch create(ExecutionCtx ctx, long nativeHandle) { - final IndicatorVector iv = new IndicatorVector(ctx, nativeHandle); + public static ColumnarBatch create(Runtime runtime, long nativeHandle) { + final IndicatorVector iv = new IndicatorVector(runtime, nativeHandle); int numColumns = Math.toIntExact(iv.getNumColumns()); int numRows = Math.toIntExact(iv.getNumRows()); if (numColumns == 0) { @@ -384,7 +384,7 @@ public static long getNativeHandle(ColumnarBatch batch) { return getIndicatorVector(batch).handle(); } - public static ExecutionCtx getExecutionCtx(ColumnarBatch batch) { - return getIndicatorVector(batch).ctx(); + public static Runtime getRuntime(ColumnarBatch batch) { + return getIndicatorVector(batch).runtime(); } } diff --git a/gluten-data/src/main/java/io/glutenproject/columnarbatch/IndicatorVector.java b/gluten-data/src/main/java/io/glutenproject/columnarbatch/IndicatorVector.java index 318746b01847..32c1f902755f 100644 --- a/gluten-data/src/main/java/io/glutenproject/columnarbatch/IndicatorVector.java +++ b/gluten-data/src/main/java/io/glutenproject/columnarbatch/IndicatorVector.java @@ -16,7 +16,7 @@ */ package io.glutenproject.columnarbatch; -import io.glutenproject.exec.ExecutionCtx; +import io.glutenproject.exec.Runtime; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Decimal; @@ -28,30 +28,30 @@ import java.util.concurrent.atomic.AtomicLong; public class IndicatorVector extends ColumnVector { - private final ExecutionCtx ctx; + private final Runtime runtime; private final long handle; private final AtomicLong refCnt = new AtomicLong(1L); - protected IndicatorVector(ExecutionCtx ctx, long handle) { + protected IndicatorVector(Runtime runtime, long handle) { super(DataTypes.NullType); - this.ctx = ctx; + this.runtime = runtime; this.handle = handle; } - public ExecutionCtx ctx() { - return ctx; + public Runtime runtime() { + return runtime; } public String getType() { - return ColumnarBatchJniWrapper.forCtx(ctx).getType(handle); + return ColumnarBatchJniWrapper.forRuntime(runtime).getType(handle); } public long getNumColumns() { - return ColumnarBatchJniWrapper.forCtx(ctx).numColumns(handle); + return ColumnarBatchJniWrapper.forRuntime(runtime).numColumns(handle); } public long getNumRows() { - return ColumnarBatchJniWrapper.forCtx(ctx).numRows(handle); + return ColumnarBatchJniWrapper.forRuntime(runtime).numRows(handle); } public long refCnt() { @@ -69,7 +69,7 @@ public void close() { return; } if (refCnt.decrementAndGet() == 0) { - ColumnarBatchJniWrapper.forCtx(ctx).close(handle); + ColumnarBatchJniWrapper.forRuntime(runtime).close(handle); } } diff --git a/gluten-data/src/main/java/io/glutenproject/datasource/DatasourceJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/datasource/DatasourceJniWrapper.java index 3ecb9679d4be..86bf7da98e13 100644 --- a/gluten-data/src/main/java/io/glutenproject/datasource/DatasourceJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/datasource/DatasourceJniWrapper.java @@ -16,9 +16,9 @@ */ package io.glutenproject.datasource; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; import io.glutenproject.init.JniUtils; import io.glutenproject.vectorized.ColumnarBatchInIterator; @@ -28,20 +28,20 @@ /** The jni file is at `cpp/core/jni/JniWrapper.cc` */ // FIXME: move to module gluten-data? -public class DatasourceJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class DatasourceJniWrapper implements RuntimeAware { + private final Runtime runtime; - private DatasourceJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private DatasourceJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static DatasourceJniWrapper create() { - return new DatasourceJniWrapper(ExecutionCtxs.contextInstance()); + return new DatasourceJniWrapper(Runtimes.contextInstance()); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } public long nativeInitDatasource( diff --git a/gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxAware.java b/gluten-data/src/main/java/io/glutenproject/exec/RuntimeAware.java similarity index 94% rename from gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxAware.java rename to gluten-data/src/main/java/io/glutenproject/exec/RuntimeAware.java index 897c16a522f4..89b4533af6e4 100644 --- a/gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxAware.java +++ b/gluten-data/src/main/java/io/glutenproject/exec/RuntimeAware.java @@ -20,6 +20,6 @@ * This defines the base abstraction for the contextual objects that can be transmitted to C++ side * for further native processing. */ -public interface ExecutionCtxAware { - long ctxHandle(); +public interface RuntimeAware { + long handle(); } diff --git a/gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/exec/RuntimeJniWrapper.java similarity index 78% rename from gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxJniWrapper.java rename to gluten-data/src/main/java/io/glutenproject/exec/RuntimeJniWrapper.java index 2b36a12a130c..d74292bca9dc 100644 --- a/gluten-data/src/main/java/io/glutenproject/exec/ExecutionCtxJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/exec/RuntimeJniWrapper.java @@ -16,11 +16,11 @@ */ package io.glutenproject.exec; -public class ExecutionCtxJniWrapper { +public class RuntimeJniWrapper { - private ExecutionCtxJniWrapper() {} + private RuntimeJniWrapper() {} - public static native long createExecutionCtx(String backendType, byte[] sessionConf); + public static native long createRuntime(String backendType, byte[] sessionConf); - public static native void releaseExecutionCtx(long handle); + public static native void releaseRuntime(long handle); } diff --git a/gluten-data/src/main/java/io/glutenproject/init/NativeBackendInitializer.java b/gluten-data/src/main/java/io/glutenproject/init/NativeBackendInitializer.java index e462735a85e0..54eac4fe9c87 100644 --- a/gluten-data/src/main/java/io/glutenproject/init/NativeBackendInitializer.java +++ b/gluten-data/src/main/java/io/glutenproject/init/NativeBackendInitializer.java @@ -36,7 +36,7 @@ public static void initializeBackend(scala.collection.Map conf) BackendsApiManager.getSettings().resolveNativeConf(nativeConfMap); initialize(JniUtils.toNativeConf(nativeConfMap)); } catch (Exception e) { - LOG.error("Error calling InitializerJniWrapper.initialize(...)", e); + LOG.error("Failed to call native backend's initialize method", e); throw e; } } diff --git a/gluten-data/src/main/java/io/glutenproject/memory/nmm/LoggingReservationListener.java b/gluten-data/src/main/java/io/glutenproject/memory/nmm/LoggingReservationListener.java new file mode 100644 index 000000000000..b4d6b586284f --- /dev/null +++ b/gluten-data/src/main/java/io/glutenproject/memory/nmm/LoggingReservationListener.java @@ -0,0 +1,59 @@ +/* + * 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 io.glutenproject.memory.nmm; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// For debugging purpose only +public class LoggingReservationListener implements ReservationListener { + private static final Logger LOGGER = LoggerFactory.getLogger(LoggingReservationListener.class); + + private final ReservationListener delegated; + + public LoggingReservationListener(ReservationListener delegated) { + this.delegated = delegated; + } + + @Override + public long reserve(long size) { + long before = getUsedBytes(); + long reserved = delegated.reserve(size); + long after = getUsedBytes(); + LOGGER.info( + String.format( + "Reservation[%s]: %d + %d(%d) = %d", this.toString(), before, reserved, size, after)); + return reserved; + } + + @Override + public long unreserve(long size) { + long before = getUsedBytes(); + long unreserved = delegated.unreserve(size); + long after = getUsedBytes(); + LOGGER.info( + String.format( + "Unreservation[%s]: %d - %d(%d) = %d", + this.toString(), before, unreserved, size, after)); + return unreserved; + } + + @Override + public long getUsedBytes() { + return delegated.getUsedBytes(); + } +} diff --git a/gluten-data/src/main/java/io/glutenproject/memory/nmm/NativeMemoryManager.java b/gluten-data/src/main/java/io/glutenproject/memory/nmm/NativeMemoryManager.java index 33a7871d5d7b..c798d292b3fc 100644 --- a/gluten-data/src/main/java/io/glutenproject/memory/nmm/NativeMemoryManager.java +++ b/gluten-data/src/main/java/io/glutenproject/memory/nmm/NativeMemoryManager.java @@ -62,6 +62,14 @@ public long shrink(long size) { return shrink(nativeInstanceHandle, size); } + // Hold this memory manager. The underlying memory pools will be released as lately as this + // memory manager gets destroyed. Which means, a call to this function would make sure the + // memory blocks directly or indirectly managed by this manager, be guaranteed safe to + // access during the period that this manager is alive. + public void hold() { + hold(nativeInstanceHandle); + } + private static native long shrink(long memoryManagerId, long size); private static native long create( @@ -75,13 +83,18 @@ private static native long create( private static native byte[] collectMemoryUsage(long memoryManagerId); + private static native void hold(long memoryManagerId); + @Override public void release() throws Exception { release(nativeInstanceHandle); if (listener.getUsedBytes() != 0) { LOGGER.warn( name - + " Reservation listener still reserved non-zero bytes, which may cause " + + " Reservation listener " + + listener.toString() + + " " + + "still reserved non-zero bytes, which may cause " + "memory leak, size: " + Utils.bytesToString(listener.getUsedBytes())); } diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchOutIterator.java b/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchOutIterator.java index 83657c7de09e..a6428d1fd871 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchOutIterator.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchOutIterator.java @@ -17,27 +17,31 @@ package io.glutenproject.vectorized; import io.glutenproject.columnarbatch.ColumnarBatches; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.memory.nmm.NativeMemoryManager; import io.glutenproject.metrics.IMetrics; import org.apache.spark.sql.vectorized.ColumnarBatch; import java.io.IOException; -public class ColumnarBatchOutIterator extends GeneralOutIterator implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class ColumnarBatchOutIterator extends GeneralOutIterator implements RuntimeAware { + private final Runtime runtime; private final long iterHandle; + private final NativeMemoryManager nmm; - public ColumnarBatchOutIterator(ExecutionCtx ctx, long iterHandle) throws IOException { + public ColumnarBatchOutIterator(Runtime runtime, long iterHandle, NativeMemoryManager nmm) + throws IOException { super(); - this.ctx = ctx; + this.runtime = runtime; this.iterHandle = iterHandle; + this.nmm = nmm; } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } @Override @@ -67,7 +71,7 @@ public ColumnarBatch nextInternal() throws IOException { if (batchHandle == -1L) { return null; // stream ended } - return ColumnarBatches.create(ctx, batchHandle); + return ColumnarBatches.create(runtime, batchHandle); } @Override @@ -81,6 +85,8 @@ public long spill(long size) { @Override public void closeInternal() { + nmm.hold(); // to make sure the outputted batches are still accessible after the iterator is + // closed nativeClose(iterHandle); } } diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchSerializerJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchSerializerJniWrapper.java index fd9c18ba2fe2..cfb8edaf0dce 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchSerializerJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/ColumnarBatchSerializerJniWrapper.java @@ -16,28 +16,28 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; -public class ColumnarBatchSerializerJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class ColumnarBatchSerializerJniWrapper implements RuntimeAware { + private final Runtime runtime; - private ColumnarBatchSerializerJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private ColumnarBatchSerializerJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static ColumnarBatchSerializerJniWrapper create() { - return new ColumnarBatchSerializerJniWrapper(ExecutionCtxs.contextInstance()); + return new ColumnarBatchSerializerJniWrapper(Runtimes.contextInstance()); } - public static ColumnarBatchSerializerJniWrapper forCtx(ExecutionCtx ctx) { - return new ColumnarBatchSerializerJniWrapper(ctx); + public static ColumnarBatchSerializerJniWrapper forRuntime(Runtime runtime) { + return new ColumnarBatchSerializerJniWrapper(runtime); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } public native ColumnarBatchSerializeResult serialize(long[] handles, long memoryManagerHandle); diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/NativeColumnarToRowJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/NativeColumnarToRowJniWrapper.java index 2e5955d7e4c6..498d95cedb00 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/NativeColumnarToRowJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/NativeColumnarToRowJniWrapper.java @@ -16,28 +16,28 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; -public class NativeColumnarToRowJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class NativeColumnarToRowJniWrapper implements RuntimeAware { + private final Runtime runtime; - private NativeColumnarToRowJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private NativeColumnarToRowJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static NativeColumnarToRowJniWrapper create() { - return new NativeColumnarToRowJniWrapper(ExecutionCtxs.contextInstance()); + return new NativeColumnarToRowJniWrapper(Runtimes.contextInstance()); } - public static NativeColumnarToRowJniWrapper forCtx(ExecutionCtx ctx) { - return new NativeColumnarToRowJniWrapper(ctx); + public static NativeColumnarToRowJniWrapper forRuntime(Runtime runtime) { + return new NativeColumnarToRowJniWrapper(runtime); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } public native long nativeColumnarToRowInit(long memoryManagerHandle) throws RuntimeException; diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/NativePlanEvaluator.java b/gluten-data/src/main/java/io/glutenproject/vectorized/NativePlanEvaluator.java index e8a3a79002b7..af4f5cf3a090 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/NativePlanEvaluator.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/NativePlanEvaluator.java @@ -17,13 +17,13 @@ package io.glutenproject.vectorized; import io.glutenproject.backendsapi.BackendsApiManager; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.Runtimes; +import io.glutenproject.memory.nmm.NativeMemoryManager; import io.glutenproject.memory.nmm.NativeMemoryManagers; import io.glutenproject.utils.DebugUtil; import io.glutenproject.validate.NativePlanValidationInfo; -import io.substrait.proto.Plan; import org.apache.spark.TaskContext; import org.apache.spark.util.SparkDirectoryUtil; @@ -37,17 +37,17 @@ public class NativePlanEvaluator { private final PlanEvaluatorJniWrapper jniWrapper; - private NativePlanEvaluator(ExecutionCtx ctx) { - jniWrapper = PlanEvaluatorJniWrapper.forCtx(ctx); + private NativePlanEvaluator(Runtime runtime) { + jniWrapper = PlanEvaluatorJniWrapper.forRuntime(runtime); } public static NativePlanEvaluator create() { - return new NativePlanEvaluator(ExecutionCtxs.contextInstance()); + return new NativePlanEvaluator(Runtimes.contextInstance()); } - public static NativePlanEvaluator createForValidation() { - // Driver side doesn't have context instance of ExecutionCtx - return new NativePlanEvaluator(ExecutionCtxs.tmpInstance()); + public static NativePlanEvaluator createForValidation(Runtime runtime) { + // Driver side doesn't have context instance of Runtime + return new NativePlanEvaluator(runtime); } public NativePlanValidationInfo doNativeValidateWithFailureReason(byte[] subPlan) { @@ -57,24 +57,24 @@ public NativePlanValidationInfo doNativeValidateWithFailureReason(byte[] subPlan // Used by WholeStageTransform to create the native computing pipeline and // return a columnar result iterator. public GeneralOutIterator createKernelWithBatchIterator( - Plan wsPlan, List iterList) throws RuntimeException, IOException { + byte[] wsPlan, List iterList) throws RuntimeException, IOException { final AtomicReference outIterator = new AtomicReference<>(); - final long memoryManagerHandle = + final NativeMemoryManager nmm = NativeMemoryManagers.create( - "WholeStageIterator", - (self, size) -> { - ColumnarBatchOutIterator instance = - Optional.of(outIterator.get()) - .orElseThrow( - () -> - new IllegalStateException( - "Fatal: spill() called before a output iterator " - + "is created. This behavior should be optimized " - + "by moving memory allocations from create() to " - + "hasNext()/next()")); - return instance.spill(size); - }) - .getNativeInstanceHandle(); + "WholeStageIterator", + (self, size) -> { + ColumnarBatchOutIterator instance = + Optional.of(outIterator.get()) + .orElseThrow( + () -> + new IllegalStateException( + "Fatal: spill() called before a output iterator " + + "is created. This behavior should be optimized " + + "by moving memory allocations from create() to " + + "hasNext()/next()")); + return instance.spill(size); + }); + final long memoryManagerHandle = nmm.getNativeInstanceHandle(); final String spillDirPath = SparkDirectoryUtil.namespace("gluten-spill") @@ -84,23 +84,19 @@ public GeneralOutIterator createKernelWithBatchIterator( long iterHandle = jniWrapper.nativeCreateKernelWithIterator( memoryManagerHandle, - getPlanBytesBuf(wsPlan), + wsPlan, iterList.toArray(new GeneralInIterator[0]), TaskContext.get().stageId(), TaskContext.getPartitionId(), TaskContext.get().taskAttemptId(), DebugUtil.saveInputToFile(), BackendsApiManager.getSparkPlanExecApiInstance().rewriteSpillPath(spillDirPath)); - outIterator.set(createOutIterator(ExecutionCtxs.contextInstance(), iterHandle)); + outIterator.set(createOutIterator(Runtimes.contextInstance(), iterHandle, nmm)); return outIterator.get(); } - private ColumnarBatchOutIterator createOutIterator(ExecutionCtx ctx, long iterHandle) - throws IOException { - return new ColumnarBatchOutIterator(ctx, iterHandle); - } - - private byte[] getPlanBytesBuf(Plan planNode) { - return planNode.toByteArray(); + private ColumnarBatchOutIterator createOutIterator( + Runtime runtime, long iterHandle, NativeMemoryManager nmm) throws IOException { + return new ColumnarBatchOutIterator(runtime, iterHandle, nmm); } } diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/NativeRowToColumnarJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/NativeRowToColumnarJniWrapper.java index f9e73e0e33e3..fa2200e0bafc 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/NativeRowToColumnarJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/NativeRowToColumnarJniWrapper.java @@ -16,24 +16,24 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; -public class NativeRowToColumnarJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class NativeRowToColumnarJniWrapper implements RuntimeAware { + private final Runtime runtime; - private NativeRowToColumnarJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private NativeRowToColumnarJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static NativeRowToColumnarJniWrapper create() { - return new NativeRowToColumnarJniWrapper(ExecutionCtxs.contextInstance()); + return new NativeRowToColumnarJniWrapper(Runtimes.contextInstance()); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } public native long init(long cSchema, long memoryManagerHandle); diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/PlanEvaluatorJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/PlanEvaluatorJniWrapper.java index cabb86c4b38a..3af3e8924c94 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/PlanEvaluatorJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/PlanEvaluatorJniWrapper.java @@ -16,9 +16,9 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; import io.glutenproject.validate.NativePlanValidationInfo; /** @@ -26,24 +26,24 @@ * This file is used to generate the .h files required for jni. Avoid all external dependencies in * this file. */ -public class PlanEvaluatorJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class PlanEvaluatorJniWrapper implements RuntimeAware { + private final Runtime runtime; - private PlanEvaluatorJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private PlanEvaluatorJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static PlanEvaluatorJniWrapper create() { - return new PlanEvaluatorJniWrapper(ExecutionCtxs.contextInstance()); + return new PlanEvaluatorJniWrapper(Runtimes.contextInstance()); } - public static PlanEvaluatorJniWrapper forCtx(ExecutionCtx ctx) { - return new PlanEvaluatorJniWrapper(ctx); + public static PlanEvaluatorJniWrapper forRuntime(Runtime runtime) { + return new PlanEvaluatorJniWrapper(runtime); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } /** @@ -54,6 +54,8 @@ public long ctxHandle() { */ native NativePlanValidationInfo nativeValidateWithFailureReason(byte[] subPlan); + public native String nativePlanString(byte[] substraitPlan, Boolean details); + /** * Create a native compute kernel and return a columnar result iterator. * @@ -70,14 +72,4 @@ public native long nativeCreateKernelWithIterator( boolean saveInputToFile, String spillDir) throws RuntimeException; - - /** Create a native compute kernel and return a row iterator. */ - native long nativeCreateKernelWithRowIterator(byte[] wsPlan) throws RuntimeException; - - /** - * Closes the projector referenced by nativeHandler. - * - * @param nativeHandler nativeHandler that needs to be closed - */ - native void nativeClose(long nativeHandler); } diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleReaderJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleReaderJniWrapper.java index 379f95cc632a..0e1f92bd2510 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleReaderJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleReaderJniWrapper.java @@ -16,32 +16,31 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; -public class ShuffleReaderJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class ShuffleReaderJniWrapper implements RuntimeAware { + private final Runtime runtime; - private ShuffleReaderJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private ShuffleReaderJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static ShuffleReaderJniWrapper create() { - return new ShuffleReaderJniWrapper(ExecutionCtxs.contextInstance()); + return new ShuffleReaderJniWrapper(Runtimes.contextInstance()); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } public native long make( long cSchema, long memoryManagerHandle, String compressionType, - String compressionCodecBackend, - String compressionMode); + String compressionCodecBackend); public native long readStream(long shuffleReaderHandle, JniByteInputStream jniIn); diff --git a/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleWriterJniWrapper.java b/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleWriterJniWrapper.java index 456b6a3f9db0..4b440c766ce7 100644 --- a/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleWriterJniWrapper.java +++ b/gluten-data/src/main/java/io/glutenproject/vectorized/ShuffleWriterJniWrapper.java @@ -16,26 +16,26 @@ */ package io.glutenproject.vectorized; -import io.glutenproject.exec.ExecutionCtx; -import io.glutenproject.exec.ExecutionCtxAware; -import io.glutenproject.exec.ExecutionCtxs; +import io.glutenproject.exec.Runtime; +import io.glutenproject.exec.RuntimeAware; +import io.glutenproject.exec.Runtimes; import java.io.IOException; -public class ShuffleWriterJniWrapper implements ExecutionCtxAware { - private final ExecutionCtx ctx; +public class ShuffleWriterJniWrapper implements RuntimeAware { + private final Runtime runtime; - private ShuffleWriterJniWrapper(ExecutionCtx ctx) { - this.ctx = ctx; + private ShuffleWriterJniWrapper(Runtime runtime) { + this.runtime = runtime; } public static ShuffleWriterJniWrapper create() { - return new ShuffleWriterJniWrapper(ExecutionCtxs.contextInstance()); + return new ShuffleWriterJniWrapper(Runtimes.contextInstance()); } @Override - public long ctxHandle() { - return ctx.getHandle(); + public long handle() { + return runtime.getHandle(); } /** diff --git a/gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtx.scala b/gluten-data/src/main/scala/io/glutenproject/exec/Runtime.scala similarity index 82% rename from gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtx.scala rename to gluten-data/src/main/scala/io/glutenproject/exec/Runtime.scala index 4c61f5ed4924..5cfa7372b3f4 100644 --- a/gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtx.scala +++ b/gluten-data/src/main/scala/io/glutenproject/exec/Runtime.scala @@ -23,9 +23,9 @@ import io.glutenproject.init.JniUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.TaskResource -class ExecutionCtx private[exec] () extends TaskResource { +class Runtime private[exec] () extends TaskResource { - private val handle = ExecutionCtxJniWrapper.createExecutionCtx( + private val handle = RuntimeJniWrapper.createRuntime( BackendsApiManager.getBackendName, JniUtils.toNativeConf( GlutenConfig.getNativeSessionConf( @@ -35,9 +35,9 @@ class ExecutionCtx private[exec] () extends TaskResource { def getHandle: Long = handle - override def release(): Unit = ExecutionCtxJniWrapper.releaseExecutionCtx(handle) + override def release(): Unit = RuntimeJniWrapper.releaseRuntime(handle) override def priority(): Int = 10 - override def resourceName(): String = s"ExecutionCtx_" + handle + override def resourceName(): String = s"Runtime_" + handle } diff --git a/gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtxs.scala b/gluten-data/src/main/scala/io/glutenproject/exec/Runtimes.scala similarity index 69% rename from gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtxs.scala rename to gluten-data/src/main/scala/io/glutenproject/exec/Runtimes.scala index 673cc1096a91..24d6cb45a885 100644 --- a/gluten-data/src/main/scala/io/glutenproject/exec/ExecutionCtxs.scala +++ b/gluten-data/src/main/scala/io/glutenproject/exec/Runtimes.scala @@ -18,24 +18,24 @@ package io.glutenproject.exec import org.apache.spark.util.TaskResources -object ExecutionCtxs { - private val EXECUTION_CTX_NAME = "ExecutionCtx" +object Runtimes { + private val RUNTIME_NAME = "Runtime" - /** Get or create the execution ctx which bound with Spark TaskContext. */ - def contextInstance(): ExecutionCtx = { + /** Get or create the runtime which bound with Spark TaskContext. */ + def contextInstance(): Runtime = { if (!TaskResources.inSparkTask()) { throw new IllegalStateException("This method must be called in a Spark task.") } - TaskResources.addResourceIfNotRegistered(EXECUTION_CTX_NAME, () => create()) + TaskResources.addResourceIfNotRegistered(RUNTIME_NAME, () => create()) } - /** Create a temporary execution ctx, caller must invoke ExecutionCtx#release manually. */ - def tmpInstance(): ExecutionCtx = { + /** Create a temporary runtime, caller must invoke Runtime#release manually. */ + def tmpInstance(): Runtime = { create() } - private def create(): ExecutionCtx = { - new ExecutionCtx + private def create(): Runtime = { + new Runtime } } diff --git a/gluten-data/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala b/gluten-data/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala index f770477f8173..0c70679eda64 100644 --- a/gluten-data/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala +++ b/gluten-data/src/main/scala/io/glutenproject/metrics/MetricsUtil.scala @@ -22,6 +22,9 @@ import io.glutenproject.substrait.{AggregationParams, JoinParams} import org.apache.spark.internal.Logging import org.apache.spark.sql.execution.SparkPlan +import java.lang.{Long => JLong} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} + object MetricsUtil extends Logging { /** @@ -38,9 +41,9 @@ object MetricsUtil extends Logging { */ def updateNativeMetrics( child: SparkPlan, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { def treeifyMetricsUpdaters(plan: SparkPlan): MetricsUpdaterTree = { plan match { case j: HashJoinLikeExecTransformer => @@ -50,7 +53,7 @@ object MetricsUtil extends Logging { case t: TransformSupport => MetricsUpdaterTree(t.metricsUpdater(), t.children.map(treeifyMetricsUpdaters)) case _ => - MetricsUpdaterTree(new NoopMetricsUpdater, Seq()) + MetricsUpdaterTree(NoopMetricsUpdater, Seq()) } } @@ -59,7 +62,7 @@ object MetricsUtil extends Logging { updateTransformerMetrics( mut, relMap, - java.lang.Long.valueOf(relMap.size() - 1), + JLong.valueOf(relMap.size() - 1), joinParamsMap, aggParamsMap) } @@ -72,8 +75,7 @@ object MetricsUtil extends Logging { * @return * the merged metrics */ - private def mergeMetrics( - operatorMetrics: java.util.ArrayList[OperatorMetrics]): OperatorMetrics = { + private def mergeMetrics(operatorMetrics: JList[OperatorMetrics]): OperatorMetrics = { if (operatorMetrics.size() == 0) { return null } @@ -165,13 +167,13 @@ object MetricsUtil extends Logging { */ def updateTransformerMetricsInternal( mutNode: MetricsUpdaterTree, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - operatorIdx: java.lang.Long, + relMap: JMap[JLong, JList[JLong]], + operatorIdx: JLong, metrics: Metrics, metricsIdx: Int, - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): (java.lang.Long, Int) = { - val operatorMetrics = new java.util.ArrayList[OperatorMetrics]() + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): (JLong, Int) = { + val operatorMetrics = new JArrayList[OperatorMetrics]() var curMetricsIdx = metricsIdx relMap .get(operatorIdx) @@ -205,7 +207,7 @@ object MetricsUtil extends Logging { u.updateNativeMetrics(opMetrics) } - var newOperatorIdx: java.lang.Long = operatorIdx - 1 + var newOperatorIdx: JLong = operatorIdx - 1 var newMetricsIdx: Int = if ( mutNode.updater.isInstanceOf[LimitMetricsUpdater] && @@ -219,7 +221,7 @@ object MetricsUtil extends Logging { mutNode.children.foreach { child => - if (!child.updater.isInstanceOf[NoopMetricsUpdater]) { + if (child.updater != NoopMetricsUpdater) { val result = updateTransformerMetricsInternal( child, relMap, @@ -256,17 +258,17 @@ object MetricsUtil extends Logging { */ def updateTransformerMetrics( mutNode: MetricsUpdaterTree, - relMap: java.util.HashMap[java.lang.Long, java.util.ArrayList[java.lang.Long]], - operatorIdx: java.lang.Long, - joinParamsMap: java.util.HashMap[java.lang.Long, JoinParams], - aggParamsMap: java.util.HashMap[java.lang.Long, AggregationParams]): IMetrics => Unit = { + relMap: JMap[JLong, JList[JLong]], + operatorIdx: JLong, + joinParamsMap: JMap[JLong, JoinParams], + aggParamsMap: JMap[JLong, AggregationParams]): IMetrics => Unit = { imetrics => try { val metrics = imetrics.asInstanceOf[Metrics] val numNativeMetrics = metrics.inputRows.length if (numNativeMetrics == 0) { () - } else if (mutNode.updater.isInstanceOf[NoopMetricsUpdater]) { + } else if (mutNode.updater == NoopMetricsUpdater) { () } else { updateTransformerMetricsInternal( diff --git a/gluten-data/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala b/gluten-data/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala deleted file mode 100644 index 021413ddaed9..000000000000 --- a/gluten-data/src/main/scala/io/glutenproject/vectorized/CloseableColumnBatchIterator.scala +++ /dev/null @@ -1,67 +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 io.glutenproject.vectorized - -import org.apache.spark.internal.Logging -import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.TaskResources - -import java.util.concurrent.TimeUnit - -/** - * An Iterator that insures that the batches [[ColumnarBatch]]s it iterates over are all closed - * properly. - */ -class CloseableColumnBatchIterator( - itr: Iterator[ColumnarBatch], - pipelineTime: Option[SQLMetric] = None) - extends Iterator[ColumnarBatch] - with Logging { - var cb: ColumnarBatch = _ - var scanTime = 0L - - override def hasNext: Boolean = { - val beforeTime = System.nanoTime() - val res = itr.hasNext - scanTime += System.nanoTime() - beforeTime - if (!res) { - pipelineTime.foreach(t => t += TimeUnit.NANOSECONDS.toMillis(scanTime)) - closeCurrentBatch() - } - res - } - - TaskResources.addRecycler("CloseableColumnBatchIterator", 100) { - closeCurrentBatch() - } - - override def next(): ColumnarBatch = { - val beforeTime = System.nanoTime() - closeCurrentBatch() - cb = itr.next() - scanTime += System.nanoTime() - beforeTime - cb - } - - private def closeCurrentBatch(): Unit = { - if (cb != null) { - cb.close() - cb = null - } - } -} diff --git a/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala b/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala index 5c5c8104dcf5..563e143d9c24 100644 --- a/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala +++ b/gluten-data/src/main/scala/io/glutenproject/vectorized/ColumnarBatchSerializer.scala @@ -17,7 +17,7 @@ package io.glutenproject.vectorized import io.glutenproject.GlutenConfig -import io.glutenproject.exec.ExecutionCtxs +import io.glutenproject.exec.Runtimes import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers import io.glutenproject.utils.ArrowAbiUtil @@ -78,6 +78,7 @@ private class ColumnarBatchSerializerInstance( extends SerializerInstance with Logging { + private lazy val nmm = NativeMemoryManagers.contextInstance("ShuffleReader") private lazy val shuffleReaderHandle = { val allocator: BufferAllocator = ArrowBufferAllocators .contextInstance() @@ -98,10 +99,9 @@ private class ColumnarBatchSerializerInstance( val jniWrapper = ShuffleReaderJniWrapper.create() val shuffleReaderHandle = jniWrapper.make( cSchema.memoryAddress(), - NativeMemoryManagers.contextInstance("ShuffleReader").getNativeInstanceHandle, + nmm.getNativeInstanceHandle, compressionCodec, - compressionCodecBackend, - GlutenConfig.getConf.columnarShuffleCompressionMode + compressionCodecBackend ) // Close shuffle reader instance as lately as the end of task processing, // since the native reader could hold a reference to memory pool that @@ -126,10 +126,11 @@ private class ColumnarBatchSerializerInstance( new DeserializationStream { private lazy val byteIn: JniByteInputStream = JniByteInputStreams.create(in) private lazy val wrappedOut: GeneralOutIterator = new ColumnarBatchOutIterator( - ExecutionCtxs.contextInstance(), + Runtimes.contextInstance(), ShuffleReaderJniWrapper .create() - .readStream(shuffleReaderHandle, byteIn)) + .readStream(shuffleReaderHandle, byteIn), + nmm) private var cb: ColumnarBatch = _ diff --git a/gluten-data/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala b/gluten-data/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala index 6526b461a07a..9e3cca7744ce 100644 --- a/gluten-data/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala +++ b/gluten-data/src/main/scala/org/apache/spark/shuffle/ColumnarShuffleWriter.scala @@ -73,7 +73,7 @@ class ColumnarShuffleWriter[K, V]( GlutenConfig.getConf.columnarShuffleCodecBackend.orNull private val bufferCompressThreshold = - GlutenConfig.getConf.columnarShuffleBufferCompressThreshold + GlutenConfig.getConf.columnarShuffleCompressionThreshold private val writeEOS = GlutenConfig.getConf.columnarShuffleWriteEOS diff --git a/gluten-data/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala b/gluten-data/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala index 293f9df7d6fe..3c90ab5ea953 100644 --- a/gluten-data/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala +++ b/gluten-data/src/main/scala/org/apache/spark/sql/execution/ColumnarBuildSideRelation.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.execution import io.glutenproject.columnarbatch.ColumnarBatches -import io.glutenproject.exec.ExecutionCtxs +import io.glutenproject.exec.Runtimes import io.glutenproject.execution.BroadCastHashJoinContext import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers -import io.glutenproject.utils.ArrowAbiUtil +import io.glutenproject.utils.{ArrowAbiUtil, Iterators} import io.glutenproject.vectorized.{ColumnarBatchSerializerJniWrapper, NativeColumnarToRowJniWrapper} import org.apache.spark.sql.catalyst.InternalRow @@ -32,7 +32,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.sql.utils.SparkArrowUtil import org.apache.spark.sql.vectorized.ColumnarBatch -import org.apache.spark.util.TaskResources import org.apache.arrow.c.ArrowSchema @@ -45,54 +44,46 @@ case class ColumnarBuildSideRelation( extends BuildSideRelation { override def deserialized: Iterator[ColumnarBatch] = { - new Iterator[ColumnarBatch] { - var batchId = 0 - var closed = false - private var finalBatch: ColumnarBatch = null - val serializeHandle: Long = { - val allocator = ArrowBufferAllocators.contextInstance() - val cSchema = ArrowSchema.allocateNew(allocator) - val arrowSchema = SparkArrowUtil.toArrowSchema( - StructType.fromAttributes(output), - SQLConf.get.sessionLocalTimeZone) - ArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema) - val handle = ColumnarBatchSerializerJniWrapper - .create() - .init( - cSchema.memoryAddress(), - NativeMemoryManagers - .contextInstance("BuildSideRelation#BatchSerializer") - .getNativeInstanceHandle) - cSchema.close() - handle - } + val serializeHandle: Long = { + val allocator = ArrowBufferAllocators.contextInstance() + val cSchema = ArrowSchema.allocateNew(allocator) + val arrowSchema = SparkArrowUtil.toArrowSchema( + StructType.fromAttributes(output), + SQLConf.get.sessionLocalTimeZone) + ArrowAbiUtil.exportSchema(allocator, arrowSchema, cSchema) + val handle = ColumnarBatchSerializerJniWrapper + .create() + .init( + cSchema.memoryAddress(), + NativeMemoryManagers + .contextInstance("BuildSideRelation#BatchSerializer") + .getNativeInstanceHandle) + cSchema.close() + handle + } - TaskResources.addRecycler(s"BuildSideRelation_deserialized_$serializeHandle", 50) { - ColumnarBatchSerializerJniWrapper.create().close(serializeHandle) - } + Iterators + .wrap(new Iterator[ColumnarBatch] { + var batchId = 0 - override def hasNext: Boolean = { - val has = batchId < batches.length - if (!has && !closed) { - if (finalBatch != null) { - ColumnarBatches.forceClose(finalBatch) - } - closed = true + override def hasNext: Boolean = { + batchId < batches.length } - has - } - override def next: ColumnarBatch = { - val handle = - ColumnarBatchSerializerJniWrapper.create().deserialize(serializeHandle, batches(batchId)) - batchId += 1 - val batch = ColumnarBatches.create(ExecutionCtxs.contextInstance(), handle) - if (batchId == batches.length) { - finalBatch = batch + override def next: ColumnarBatch = { + val handle = + ColumnarBatchSerializerJniWrapper + .create() + .deserialize(serializeHandle, batches(batchId)) + batchId += 1 + ColumnarBatches.create(Runtimes.contextInstance(), handle) } - batch + }) + .recycleIterator { + ColumnarBatchSerializerJniWrapper.create().close(serializeHandle) } - } + .recyclePayload(ColumnarBatches.forceClose) // FIXME why force close? + .create() } override def asReadOnlyCopy( @@ -104,9 +95,9 @@ case class ColumnarBuildSideRelation( */ override def transform(key: Expression): Array[InternalRow] = { // This transformation happens in Spark driver, thus resources can not be managed automatically. - val executionCtx = ExecutionCtxs.tmpInstance() + val runtime = Runtimes.tmpInstance() val nativeMemoryManager = NativeMemoryManagers.tmpInstance("BuildSideRelation#transform") - val serializerJniWrapper = ColumnarBatchSerializerJniWrapper.forCtx(executionCtx) + val serializerJniWrapper = ColumnarBatchSerializerJniWrapper.forRuntime(runtime) val serializeHandle = { val allocator = ArrowBufferAllocators.globalInstance() val cSchema = ArrowSchema.allocateNew(allocator) @@ -123,7 +114,7 @@ case class ColumnarBuildSideRelation( var closed = false // Convert columnar to Row. - val jniWrapper = NativeColumnarToRowJniWrapper.forCtx(executionCtx) + val jniWrapper = NativeColumnarToRowJniWrapper.forRuntime(runtime) val c2rId = jniWrapper.nativeColumnarToRowInit(nativeMemoryManager.getNativeInstanceHandle) var batchId = 0 val iterator = if (batches.length > 0) { @@ -133,7 +124,7 @@ case class ColumnarBuildSideRelation( if (!itHasNext && !closed) { jniWrapper.nativeClose(c2rId) serializerJniWrapper.close(serializeHandle) - executionCtx.release() + runtime.release() nativeMemoryManager.release() closed = true } @@ -145,7 +136,7 @@ case class ColumnarBuildSideRelation( batchId += 1 val batchHandle = serializerJniWrapper.deserialize(serializeHandle, batchBytes) - val batch = ColumnarBatches.create(executionCtx, batchHandle) + val batch = ColumnarBatches.create(runtime, batchHandle) if (batch.numRows == 0) { batch.close() Iterator.empty diff --git a/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala b/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala index 214ddb497c47..eb29ba2709a0 100644 --- a/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala +++ b/gluten-data/src/main/scala/org/apache/spark/sql/execution/utils/ExecUtil.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.execution.utils import io.glutenproject.columnarbatch.ColumnarBatches import io.glutenproject.memory.arrowalloc.ArrowBufferAllocators import io.glutenproject.memory.nmm.NativeMemoryManagers +import io.glutenproject.utils.Iterators import io.glutenproject.vectorized.{ArrowWritableColumnVector, NativeColumnarToRowInfo, NativeColumnarToRowJniWrapper, NativePartitioning} import org.apache.spark.{Partitioner, RangePartitioner, ShuffleDependency} -import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ColumnarShuffleDependency @@ -30,13 +30,12 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.PartitionIdPassthrough import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} -import org.apache.spark.util.{MutablePair, TaskResources} +import org.apache.spark.util.MutablePair object ExecUtil { @@ -50,35 +49,27 @@ object ExecUtil { .getNativeInstanceHandle) info = jniWrapper.nativeColumnarToRowConvert(batchHandle, c2rHandle) - new Iterator[InternalRow] { - var rowId = 0 - val row = new UnsafeRow(batch.numCols()) - var closed = false + Iterators + .wrap(new Iterator[InternalRow] { + var rowId = 0 + val row = new UnsafeRow(batch.numCols()) - TaskResources.addRecycler(s"ColumnarToRow_$c2rHandle", 100) { - if (!closed) { - jniWrapper.nativeClose(c2rHandle) - closed = true + override def hasNext: Boolean = { + rowId < batch.numRows() } - } - override def hasNext: Boolean = { - val result = rowId < batch.numRows() - if (!result && !closed) { - jniWrapper.nativeClose(c2rHandle) - closed = true + override def next: UnsafeRow = { + if (rowId >= batch.numRows()) throw new NoSuchElementException + val (offset, length) = (info.offsets(rowId), info.lengths(rowId)) + row.pointTo(null, info.memoryAddress + offset, length.toInt) + rowId += 1 + row } - result - } - - override def next: UnsafeRow = { - if (rowId >= batch.numRows()) throw new NoSuchElementException - val (offset, length) = (info.offsets(rowId), info.lengths(rowId)) - row.pointTo(null, info.memoryAddress + offset, length.toInt) - rowId += 1 - row + }) + .recycleIterator { + jniWrapper.nativeClose(c2rHandle) } - } + .create() } // scalastyle:off argcount @@ -126,29 +117,31 @@ object ExecUtil { // only used for fallback range partitioning def computeAndAddPartitionId( cbIter: Iterator[ColumnarBatch], - partitionKeyExtractor: InternalRow => Any): CloseablePairedColumnarBatchIterator = { - CloseablePairedColumnarBatchIterator { - cbIter - .filter(cb => cb.numRows != 0 && cb.numCols != 0) - .map { - cb => - val pidVec = ArrowWritableColumnVector - .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType)) - .head - convertColumnarToRow(cb).zipWithIndex.foreach { - case (row, i) => - val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row)) - pidVec.putInt(i, pid) - } - val pidBatch = ColumnarBatches.ensureOffloaded( - ArrowBufferAllocators.contextInstance(), - new ColumnarBatch(Array[ColumnVector](pidVec), cb.numRows)) - val newHandle = ColumnarBatches.compose(pidBatch, cb) - // Composed batch already hold pidBatch's shared ref, so close is safe. - ColumnarBatches.forceClose(pidBatch) - (0, ColumnarBatches.create(ColumnarBatches.getExecutionCtx(cb), newHandle)) - } - } + partitionKeyExtractor: InternalRow => Any): Iterator[(Int, ColumnarBatch)] = { + Iterators + .wrap( + cbIter + .filter(cb => cb.numRows != 0 && cb.numCols != 0) + .map { + cb => + val pidVec = ArrowWritableColumnVector + .allocateColumns(cb.numRows, new StructType().add("pid", IntegerType)) + .head + convertColumnarToRow(cb).zipWithIndex.foreach { + case (row, i) => + val pid = rangePartitioner.get.getPartition(partitionKeyExtractor(row)) + pidVec.putInt(i, pid) + } + val pidBatch = ColumnarBatches.ensureOffloaded( + ArrowBufferAllocators.contextInstance(), + new ColumnarBatch(Array[ColumnVector](pidVec), cb.numRows)) + val newHandle = ColumnarBatches.compose(pidBatch, cb) + // Composed batch already hold pidBatch's shared ref, so close is safe. + ColumnarBatches.forceClose(pidBatch) + (0, ColumnarBatches.create(ColumnarBatches.getRuntime(cb), newHandle)) + }) + .recyclePayload(p => ColumnarBatches.forceClose(p._2)) // FIXME why force close? + .create() } val nativePartitioning: NativePartitioning = newPartitioning match { @@ -182,11 +175,6 @@ object ExecUtil { row => projection(row) } val newIter = computeAndAddPartitionId(cbIter, partitionKeyExtractor) - - TaskResources.addRecycler("RangePartitioningIter", 100) { - newIter.closeColumnBatch() - } - newIter }, isOrderSensitive = isOrderSensitive @@ -210,35 +198,6 @@ object ExecUtil { dependency } } - -case class CloseablePairedColumnarBatchIterator(iter: Iterator[(Int, ColumnarBatch)]) - extends Iterator[(Int, ColumnarBatch)] - with Logging { - - private var cur: (Int, ColumnarBatch) = _ - - override def hasNext: Boolean = { - iter.hasNext - } - - override def next(): (Int, ColumnarBatch) = { - closeColumnBatch() - if (iter.hasNext) { - cur = iter.next() - cur - } else { - closeColumnBatch() - Iterator.empty.next() - } - } - - def closeColumnBatch(): Unit = { - if (cur != null) { - logDebug("Close appended partition id vector") - cur match { - case (_, cb: ColumnarBatch) => ColumnarBatches.forceClose(cb) - } - cur = null - } - } +private[spark] class PartitionIdPassthrough(override val numPartitions: Int) extends Partitioner { + override def getPartition(key: Any): Int = key.asInstanceOf[Int] } diff --git a/gluten-delta/pom.xml b/gluten-delta/pom.xml new file mode 100755 index 000000000000..76e64e9e3b4c --- /dev/null +++ b/gluten-delta/pom.xml @@ -0,0 +1,155 @@ + + + + gluten-parent + io.glutenproject + 1.1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + gluten-delta + jar + Gluten DeltaLake + + + ${project.basedir}/src/main/resources + + + + + io.glutenproject + gluten-core + ${project.version} + provided + + + io.delta + delta-core_${scala.binary.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + + + + + io.glutenproject + gluten-core + ${project.version} + test-jar + test + + + io.glutenproject + backends-velox + ${project.version} + test + + + io.glutenproject + backends-velox + ${project.version} + test-jar + test + + + org.apache.spark + spark-core_${scala.binary.version} + + + org.apache.spark + spark-core_${scala.binary.version} + test-jar + + + org.apache.spark + spark-sql_${scala.binary.version} + test-jar + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + test + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + ${resource.dir} + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + test-compile + + test-jar + + + + + + org.apache.maven.plugins + maven-resources-plugin + + + + diff --git a/gluten-delta/src/main/resources/META-INF/services/io.glutenproject.extension.RewriteTransformerRules b/gluten-delta/src/main/resources/META-INF/services/io.glutenproject.extension.RewriteTransformerRules new file mode 100644 index 000000000000..d4a43f34613d --- /dev/null +++ b/gluten-delta/src/main/resources/META-INF/services/io.glutenproject.extension.RewriteTransformerRules @@ -0,0 +1 @@ +io.glutenproject.extension.DeltaRewriteTransformerRules \ No newline at end of file diff --git a/gluten-delta/src/main/scala/io/glutenproject/extension/DeltaRewriteTransformerRules.scala b/gluten-delta/src/main/scala/io/glutenproject/extension/DeltaRewriteTransformerRules.scala new file mode 100644 index 000000000000..047454bb61db --- /dev/null +++ b/gluten-delta/src/main/scala/io/glutenproject/extension/DeltaRewriteTransformerRules.scala @@ -0,0 +1,163 @@ +/* + * 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 io.glutenproject.extension + +import io.glutenproject.execution.{FileSourceScanExecTransformer, ProjectExecTransformer} +import io.glutenproject.extension.DeltaRewriteTransformerRules.columnMappingRule +import io.glutenproject.extension.columnar.TransformHints + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag +import org.apache.spark.sql.delta.{DeltaParquetFileFormat, NoMapping} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.types.{StructField, StructType} + +import scala.collection._ + +class DeltaRewriteTransformerRules extends RewriteTransformerRules { + override def rules: Seq[Rule[SparkPlan]] = columnMappingRule :: Nil +} + +object DeltaRewriteTransformerRules { + + private val COLUMN_MAPPING_RULE_TAG: TreeNodeTag[String] = + TreeNodeTag[String]("io.glutenproject.delta.column.mapping") + + private def notAppliedColumnMappingRule(plan: SparkPlan): Boolean = { + plan.getTagValue(COLUMN_MAPPING_RULE_TAG).isEmpty + } + + private def tagColumnMappingRule(plan: SparkPlan): Unit = { + plan.setTagValue(COLUMN_MAPPING_RULE_TAG, null) + } + + val columnMappingRule: Rule[SparkPlan] = (plan: SparkPlan) => + plan.transformWithSubqueries { + // If it enables Delta Column Mapping(e.g. nameMapping and idMapping), + // transform the metadata of Delta into Parquet's, + // so that gluten can read Delta File using Parquet Reader. + case p: FileSourceScanExecTransformer + if isDeltaColumnMappingFileFormat(p.relation.fileFormat) && notAppliedColumnMappingRule( + p) => + transformColumnMappingPlan(p) + } + + private def isDeltaColumnMappingFileFormat(fileFormat: FileFormat): Boolean = fileFormat match { + case d: DeltaParquetFileFormat if d.columnMappingMode != NoMapping => + true + case _ => + false + } + + /** + * This method is only used for Delta ColumnMapping FileFormat(e.g. nameMapping and idMapping) + * transform the metadata of Delta into Parquet's, each plan should only be transformed once. + */ + private def transformColumnMappingPlan(plan: SparkPlan): SparkPlan = plan match { + case plan: FileSourceScanExecTransformer => + val fmt = plan.relation.fileFormat.asInstanceOf[DeltaParquetFileFormat] + // a mapping between the table schemas name to parquet schemas. + val columnNameMapping = mutable.Map.empty[String, String] + fmt.referenceSchema.foreach { + f => + val pName = f.metadata.getString("delta.columnMapping.physicalName") + val lName = f.name + columnNameMapping += (lName -> pName) + } + + // transform HadoopFsRelation + val relation = plan.relation + val newDataFields = relation.dataSchema.map(e => e.copy(columnNameMapping(e.name))) + val newPartitionFields = relation.partitionSchema.map { + e => e.copy(columnNameMapping(e.name)) + } + val newFsRelation = relation.copy( + partitionSchema = StructType(newPartitionFields), + dataSchema = StructType(newDataFields) + )(SparkSession.active) + + // transform output's name into physical name so Reader can read data correctly + // should keep the columns order the same as the origin output + val originColumnNames = mutable.ListBuffer.empty[String] + val transformedAttrs = mutable.ListBuffer.empty[Attribute] + val newOutput = plan.output.map { + o => + val newAttr = o.withName(columnNameMapping(o.name)) + if (!originColumnNames.contains(o.name)) { + transformedAttrs += newAttr + originColumnNames += o.name + } + newAttr + } + // transform dataFilters + val newDataFilters = plan.dataFilters.map { + e => + e.transformDown { + case attr: AttributeReference => + val newAttr = attr.withName(columnNameMapping(attr.name)).toAttribute + if (!originColumnNames.contains(attr.name)) { + transformedAttrs += newAttr + originColumnNames += attr.name + } + newAttr + } + } + // transform partitionFilters + val newPartitionFilters = plan.partitionFilters.map { + e => + e.transformDown { + case attr: AttributeReference => + val newAttr = attr.withName(columnNameMapping(attr.name)).toAttribute + if (!originColumnNames.contains(attr.name)) { + transformedAttrs += newAttr + originColumnNames += attr.name + } + newAttr + } + } + // replace tableName in schema with physicalName + val newRequiredFields = plan.requiredSchema.map { + e => StructField(columnNameMapping(e.name), e.dataType, e.nullable, e.metadata) + } + val scanExecTransformer = new FileSourceScanExecTransformer( + newFsRelation, + newOutput, + StructType(newRequiredFields), + newPartitionFilters, + plan.optionalBucketSet, + plan.optionalNumCoalescedBuckets, + newDataFilters, + plan.tableIdentifier, + plan.disableBucketedScan + ) + scanExecTransformer.copyTagsFrom(plan) + tagColumnMappingRule(scanExecTransformer) + TransformHints.tagTransformable(scanExecTransformer) + + // alias physicalName into tableName + val expr = (transformedAttrs, originColumnNames).zipped.map { + (attr, columnName) => Alias(attr, columnName)(exprId = attr.exprId) + } + val projectExecTransformer = ProjectExecTransformer(expr, scanExecTransformer) + TransformHints.tagTransformable(projectExecTransformer) + projectExecTransformer + case _ => plan + } +} diff --git a/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.scala new file mode 100644 index 000000000000..3f1f7ab255fa --- /dev/null +++ b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxDeltaSuite.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 io.glutenproject.execution + +import org.apache.spark.SparkConf +import org.apache.spark.sql.Row + +class VeloxDeltaSuite extends WholeStageTransformerSuite { + + protected val rootPath: String = getClass.getResource("/").getPath + override protected val backend: String = "velox" + override protected val resourcePath: String = "/tpch-data-parquet-velox" + override protected val fileFormat: String = "parquet" + + override protected def sparkConf: SparkConf = { + 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.unsafe.exceptionOnMemoryLeak", "true") + .set("spark.sql.autoBroadcastJoinThreshold", "-1") + .set("spark.sql.sources.useV1SourceList", "avro") + .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + } + + test("column mapping mode") { + spark.sql(s""" + |create table delta_cm1 (id int, name string) using delta + |tblproperties ("delta.columnMapping.mode"= "id") + |""".stripMargin) + spark.sql(s""" + |insert into delta_cm1 values (1, "v1"), (2, "v2") + |""".stripMargin) + val df1 = runQueryAndCompare("select * from delta_cm1") { _ => } + checkLengthAndPlan(df1, 2) + checkAnswer(df1, Row(1, "v1") :: Row(2, "v2") :: Nil) + + val df2 = runQueryAndCompare("select name from delta_cm1 where id = 2") { _ => } + checkLengthAndPlan(df2, 1) + checkAnswer(df2, Row("v2") :: Nil) + } +} diff --git a/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxTPCHDeltaSuite.scala b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxTPCHDeltaSuite.scala new file mode 100644 index 000000000000..fe976e449bdf --- /dev/null +++ b/gluten-delta/src/test/scala/io/glutenproject/execution/VeloxTPCHDeltaSuite.scala @@ -0,0 +1,52 @@ +/* + * 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 io.glutenproject.execution + +import org.apache.spark.SparkConf + +import java.io.File + +class VeloxTPCHDeltaSuite extends VeloxTPCHSuite { + + protected val tpchBasePath: String = new File( + "../backends-velox/src/test/resources").getAbsolutePath + + override protected val resourcePath: String = + new File(tpchBasePath, "tpch-data-parquet-velox").getCanonicalPath + + override protected val veloxTPCHQueries: String = + new File(tpchBasePath, "tpch-queries-velox").getCanonicalPath + + override protected val queriesResults: String = + new File(tpchBasePath, "queries-output").getCanonicalPath + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") + .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") + } + + override protected def createTPCHNotNullTables(): Unit = { + TPCHTables = TPCHTableNames.map { + table => + val tablePath = new File(resourcePath, table).getAbsolutePath + val tableDF = spark.read.format(fileFormat).load(tablePath) + tableDF.write.format("delta").mode("append").saveAsTable(table) + (table, tableDF) + }.toMap + } +} diff --git a/gluten-ut/common/src/test/scala/io/glutenproject/utils/BackendTestSettings.scala b/gluten-ut/common/src/test/scala/io/glutenproject/utils/BackendTestSettings.scala index 28af55e3106a..af33a75be179 100644 --- a/gluten-ut/common/src/test/scala/io/glutenproject/utils/BackendTestSettings.scala +++ b/gluten-ut/common/src/test/scala/io/glutenproject/utils/BackendTestSettings.scala @@ -25,16 +25,16 @@ import scala.reflect.ClassTag abstract class BackendTestSettings { - private val enabledSuites: java.util.Map[String, TestNameFilters] = new util.HashMap() + private val enabledSuites: java.util.Map[String, SuiteSettings] = new util.HashMap() - protected def enableSuite[T: ClassTag]: TestNameFilters = { + protected def enableSuite[T: ClassTag]: SuiteSettings = { val suiteName = implicitly[ClassTag[T]].runtimeClass.getCanonicalName if (enabledSuites.containsKey(suiteName)) { throw new IllegalArgumentException("Duplicated suite name: " + suiteName) } - val filters = new TestNameFilters - enabledSuites.put(suiteName, filters) - filters + val suiteSettings = new SuiteSettings + enabledSuites.put(suiteName, suiteSettings) + suiteSettings } private[utils] def shouldRun(suiteName: String, testName: String): Boolean = { @@ -42,10 +42,15 @@ abstract class BackendTestSettings { return false } - val filters = enabledSuites.get(suiteName) + val suiteSettings = enabledSuites.get(suiteName) - val inclusion = filters.inclusion.asScala - val exclusion = filters.exclusion.asScala + suiteSettings.disableReason match { + case Some(_) => return false + case _ => // continue + } + + val inclusion = suiteSettings.inclusion.asScala + val exclusion = suiteSettings.exclusion.asScala if (inclusion.isEmpty && exclusion.isEmpty) { // default to run all cases under this suite @@ -73,26 +78,36 @@ abstract class BackendTestSettings { throw new IllegalStateException("Unreachable code") } - final protected class TestNameFilters { + final protected class SuiteSettings { private[utils] val inclusion: util.List[IncludeBase] = new util.ArrayList() private[utils] val exclusion: util.List[ExcludeBase] = new util.ArrayList() - def include(testNames: String*): TestNameFilters = { + private[utils] var disableReason: Option[String] = None + + def include(testNames: String*): SuiteSettings = { inclusion.add(Include(testNames: _*)) this } - def exclude(testNames: String*): TestNameFilters = { + def exclude(testNames: String*): SuiteSettings = { exclusion.add(Exclude(testNames: _*)) this } - def includeByPrefix(prefixes: String*): TestNameFilters = { + def includeByPrefix(prefixes: String*): SuiteSettings = { inclusion.add(IncludeByPrefix(prefixes: _*)) this } - def excludeByPrefix(prefixes: String*): TestNameFilters = { + def excludeByPrefix(prefixes: String*): SuiteSettings = { exclusion.add(ExcludeByPrefix(prefixes: _*)) this } + + def disableByReason(reason: String): SuiteSettings = { + disableReason = disableReason match { + case Some(r) => throw new IllegalArgumentException("Disable reason already set: " + r) + case None => Some(reason) + } + this + } } protected trait IncludeBase { diff --git a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala index 7ca393f7666b..fa8c146d35a6 100644 --- a/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala +++ b/gluten-ut/common/src/test/scala/org/apache/spark/sql/GlutenTestsTrait.scala @@ -117,6 +117,7 @@ trait GlutenTestsTrait extends GlutenTestsCommonTrait { .config(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) .config("spark.unsafe.exceptionOnMemoryLeak", "true") .config(GlutenConfig.UT_STATISTIC.key, "true") + .config("spark.sql.decimalOperations.allowPrecisionLoss", "false") .getOrCreate() } else { sparkBuilder diff --git a/gluten-ut/pom.xml b/gluten-ut/pom.xml index e887e88f06fe..75d80f3eda3d 100644 --- a/gluten-ut/pom.xml +++ b/gluten-ut/pom.xml @@ -24,6 +24,10 @@ ../pom.xml + + common + + gluten-ut pom Gluten Unit Test @@ -54,7 +58,7 @@ org.apache.spark - spark-hive-thriftserver_${scala.binary.version} + spark-hive_${scala.binary.version} provided @@ -174,27 +178,25 @@ - spark-3.2 - - true - - - spark32 - common spark-3.3 spark33 - common + + + + spark-3.4 + + spark34 diff --git a/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala index 2b9f5d66e458..dc15e2fee1c2 100644 --- a/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala @@ -287,6 +287,10 @@ class ClickHouseTestSettings extends BackendTestSettings { "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan].exclude( "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOnDisableProject].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOffDisableProject].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff].exclude( "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan].exclude( @@ -295,6 +299,10 @@ class ClickHouseTestSettings extends BackendTestSettings { "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan].exclude( "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOnDisableProject].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOffDisableProject].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") enableSuite[GlutenExpressionsSchemaSuite] enableSuite[GlutenExtraStrategiesSuite] enableSuite[GlutenFileBasedDataSourceSuite] @@ -565,6 +573,7 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-32828: cast from a derived user-defined type to a base type") .exclude("SPARK-34727: cast from float II") .exclude("SPARK-35720: cast invalid string input to timestamp without time zone") + .exclude("Cast should output null for invalid strings when ANSI is not enabled.") enableSuite[GlutenCastSuiteWithAnsiModeOn] .exclude("null cast") .exclude("cast string to date") @@ -669,6 +678,7 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("WeekDay") .exclude("WeekOfYear") .exclude("DateFormat") + .exclude("Gluten - DateFormat") .exclude("Hour") .exclude("Minute") .exclude("date add interval") @@ -1666,7 +1676,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("Legacy mode - nested map with struct as key type") .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") .exclude("SPARK-35640: int as long should throw schema incompatible error") - .exclude("Gluten - SPARK-35640: int as long should throw schema incompatible error") enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") enableSuite[GlutenParquetRebaseDatetimeV1Suite] diff --git a/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala index 39c7310b5ada..def4ed70681e 100644 --- a/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark32/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala @@ -125,6 +125,8 @@ class VeloxTestSettings extends BackendTestSettings { // Not supported for approx_count_distinct "SPARK-34165: Add count_distinct to summary" ) + // Double precision loss: https://github.com/facebookincubator/velox/pull/6051#issuecomment-1731028215. + .exclude("SPARK-22271: mean overflows and returns null for some decimal variables") enableSuite[GlutenDataFrameNaFunctionsSuite] .exclude( @@ -137,10 +139,14 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOnDisableProject] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOffDisableProject] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOnDisableProject] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOffDisableProject] enableSuite[GlutenAdaptiveQueryExecSuite] .includeByPrefix( @@ -201,6 +207,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("to_unix_timestamp") // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenStringFunctionsSuite] enableSuite[GlutenRegexpExpressionsSuite] @@ -261,6 +269,12 @@ class VeloxTestSettings extends BackendTestSettings { "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. ) enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .exclude("unbounded preceding/following range between with aggregation") + .exclude("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .exclude("rows between should accept int/long values as boundary") enableSuite[GlutenColumnExpressionSuite] enableSuite[GlutenDataFrameImplicitsSuite] enableSuite[GlutenGeneratorFunctionSuite] @@ -348,13 +362,17 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Return correct results when data columns overlap with partition " + "columns (nested data)") .exclude("SPARK-31116: Select nested schema with case insensitive mode") + .exclude("SPARK-23271 empty RDD when saved should write a metadata only file - orc") + .exclude("SPARK-22146 read files containing special characters using orc") + .exclude("Do not use cache on overwrite") + .exclude("Do not use cache on append") + .exclude("File source v2: support partition pruning") + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") enableSuite[GlutenEnsureRequirementsSuite] // Rewrite to change the shuffle partitions for optimizing repartition .excludeByPrefix("SPARK-35675") enableSuite[GlutenCoalesceShufflePartitionsSuite] - // Rewrite with columnar operators .excludeByPrefix("SPARK-24705") - .excludeByPrefix("SPARK-34790") .excludeByPrefix("determining the number of reducers") enableSuite[GlutenFileSourceCharVarcharTestSuite] enableSuite[GlutenDSV2CharVarcharTestSuite] @@ -768,6 +786,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] enableSuite[GlutenParquetEncodingSuite] enableSuite[GlutenParquetFileFormatV1Suite] @@ -777,6 +797,8 @@ class VeloxTestSettings extends BackendTestSettings { .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. + .exclude("filter pushdown - timestamp") .exclude("filter pushdown - date") // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") @@ -791,6 +813,8 @@ class VeloxTestSettings extends BackendTestSettings { .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. + .exclude("filter pushdown - timestamp") .exclude("filter pushdown - date") // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") @@ -817,7 +841,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34817: Read UINT_8/UINT_16/UINT_32 from parquet") // Exception. .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") - // Rewrite to align exception msg. + // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") // Timestamp is read as INT96. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala index e8a424de5be1..d0d856b5167b 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala @@ -16,4 +16,11 @@ */ package org.apache.spark.sql -class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait {} +class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Cancelling stage in a query with Range", + "SPARK-20430 Initialize Range parameters in a driver side" + ) + +} diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala index 4ac18a1cc629..6510f2f467d3 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala @@ -721,6 +721,22 @@ class GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan } } +class GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOnDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOffDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + } +} + // Test DPP with batch scan disabled by user for some reason, which can also mock the situation // that scan is not transformable. class GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan @@ -737,3 +753,19 @@ class GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") } } + +class GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOnDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + } +} + +class GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOffDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index f27326176de0..24a44b802d68 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -212,7 +212,8 @@ class GlutenSQLQueryTestSuite "explain.sql", // explain "group-analytics.sql", // wait velox to fix issue 3357 "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql" // blocked by VELOX-5768 + "higher-order-functions.sql", // blocked by VELOX-5768 + "udf/udf-window.sql" // Local window fixes are not added. ) ++ otherIgnoreList /** @@ -239,8 +240,9 @@ class GlutenSQLQueryTestSuite "current_database_catalog.sql", "date.sql", "datetime-formatting-invalid.sql", - "datetime-formatting-legacy.sql", - "datetime-formatting.sql", + // Velox had different handling for some illegal cases. +// "datetime-formatting-legacy.sql", +// "datetime-formatting.sql", "datetime-legacy.sql", "datetime-parsing-invalid.sql", "datetime-parsing-legacy.sql", diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala index 79e26e336ab2..fcc8912996bb 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, TimeZoneUTC} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DateType, IntegerType, StringType} +import org.apache.spark.sql.types.{DateType, IntegerType, StringType, TimestampType} import org.apache.spark.unsafe.types.UTF8String import java.sql.{Date, Timestamp} @@ -285,4 +285,62 @@ class GlutenDateExpressionsSuite extends DateExpressionsSuite with GlutenTestsTr GenerateUnsafeProjection.generate( ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } + + // Modified based on vanilla spark to explicitly set timezone in config. + test(GlutenTestConstants.GLUTEN_TEST + "DateFormat") { + val PST_OPT = Option(PST.getId) + val JST_OPT = Option(JST.getId) + + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), + null) + checkEvaluation( + DateFormatClass( + Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), + UTC_OPT), + null) + + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("y"), UTC_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("H"), UTC_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("y"), PST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("H"), PST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("y"), JST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("H"), JST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + } + } + } } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala index 4489d647e2b4..6ae24716ca9d 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, Row, SparkSession} import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ColumnarAQEShuffleReadExec, QueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec, ShuffleQueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.functions.{col, max} import org.apache.spark.sql.internal.SQLConf @@ -30,12 +30,6 @@ class GlutenCoalesceShufflePartitionsSuite extends CoalesceShufflePartitionsSuite with GlutenTestsCommonTrait { - object ColumnarCoalescedShuffleRead { - def unapply(read: ColumnarAQEShuffleReadExec): Boolean = { - !read.isLocalRead && !read.hasSkewedPartition && read.hasCoalescedPartition - } - } - override protected def afterAll(): Unit = {} override def withSparkSession( @@ -99,10 +93,7 @@ class GlutenCoalesceShufflePartitionsSuite assert(finalPlan.collect { case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r }.length == 2) - assert(finalPlan.collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - }.length == 3) + assert(finalPlan.collect { case r @ CoalescedShuffleRead() => r }.length == 3) // test case 2: a query stage has 2 parent stages. // Final Stage 3 @@ -133,10 +124,7 @@ class GlutenCoalesceShufflePartitionsSuite level1Stages.foreach( qs => assert( - qs.plan.collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - }.length == 1, + qs.plan.collect { case r @ CoalescedShuffleRead() => r }.length == 1, "Wrong CoalescedShuffleRead below " + qs.simpleString(3) )) @@ -158,31 +146,6 @@ class GlutenCoalesceShufflePartitionsSuite withSparkSession(test, 400, None) } - test(GLUTEN_TEST + "SPARK-34790: enable IO encryption in AQE partition coalescing") { - val test: SparkSession => Unit = { - spark: SparkSession => - val ds = spark.range(0, 100, 1, numInputPartitions) - val resultDf = ds.repartition(ds.col("id")) - resultDf.collect() - - val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec] - .executedPlan - assert( - finalPlan - .collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - } - .isDefinedAt(0)) - } - Seq(true, false).foreach { - enableIOEncryption => - // Before SPARK-34790, it will throw an exception when io encryption enabled. - withSparkSession(test, Int.MaxValue, None, enableIOEncryption) - } - } - Seq(Some(5), None).foreach { minNumPostShufflePartitions => val testNameNote = minNumPostShufflePartitions match { @@ -208,11 +171,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = agg.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -255,11 +214,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -267,7 +222,7 @@ class GlutenCoalesceShufflePartitionsSuite case None => assert(shuffleReads.length === 2) - shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) } } // Change the original value 16384 to 40000 for gluten. The test depends on the calculation @@ -308,11 +263,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -320,7 +271,7 @@ class GlutenCoalesceShufflePartitionsSuite case None => assert(shuffleReads.length === 2) - shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) } } @@ -362,11 +313,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -374,7 +321,7 @@ class GlutenCoalesceShufflePartitionsSuite case None => assert(shuffleReads.length === 2) - shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 4)) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) } } @@ -411,11 +358,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } assert(shuffleReads.length === 0) } finally { spark.sql("drop table t") diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala index db78728790f7..3ac53799f3f9 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala @@ -146,7 +146,6 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length val numLocalReads = collect(plan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read case r: AQEShuffleReadExec if r.isLocalRead => r } // because columnar local reads cannot execute @@ -207,7 +206,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL assert(sortMergeJoinSize(plan) == 1) assert(broadcastHashJoinSize(adaptivePlan) == 1) val localReads = collect(adaptivePlan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read + case read: AQEShuffleReadExec if read.isLocalRead => read } assert(localReads.length == 2) } @@ -225,14 +224,14 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) assert(bhj.size == 1) val localReads = collect(adaptivePlan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read + case read: AQEShuffleReadExec if read.isLocalRead => read } assert(localReads.length == 2) val localShuffleRDD0 = localReads(0) - .doExecuteColumnar() + .executeColumnar() .asInstanceOf[ShuffledColumnarBatchRDD] val localShuffleRDD1 = localReads(1) - .doExecuteColumnar() + .executeColumnar() .asInstanceOf[ShuffledColumnarBatchRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 // and the partitions length is 2 * numMappers = 4 @@ -271,7 +270,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL checkAnswer(testDf, Seq()) val plan = testDf.queryExecution.executedPlan assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformer]).isDefined) - val coalescedReads = collect(plan) { case r: ColumnarAQEShuffleReadExec => r } + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } assert(coalescedReads.length == 3, s"$plan") coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) } @@ -438,8 +437,10 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL test("gluten Exchange reuse") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "100", - SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + // magic threshold, ch backend has two bhj when threshold is 100 + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "90", + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT value FROM testData join testData2 ON key = a " + "join (SELECT value v from testData join testData3 ON key = a) on value = v") @@ -673,7 +674,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL } assert( joins.head.left - .collect { case r: ColumnarAQEShuffleReadExec => r } + .collect { case r: AQEShuffleReadExec => r } .head .partitionSpecs .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } @@ -681,7 +682,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL .length == leftSkewNum) assert( joins.head.right - .collect { case r: ColumnarAQEShuffleReadExec => r } + .collect { case r: AQEShuffleReadExec => r } .head .partitionSpecs .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } @@ -717,10 +718,10 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL test("gluten SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") - val reads = collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head - val c = read.canonicalized.asInstanceOf[ColumnarAQEShuffleReadExec] + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] // we can't just call execute() because that has separate checks for canonicalized plans val ex = intercept[IllegalStateException] { val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) @@ -735,7 +736,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.SHUFFLE_PARTITIONS.key -> "5") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") - val reads = collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head assert(!read.isLocalRead) @@ -756,7 +757,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformer => j }.head assert(join.joinBuildSide == BuildLeft) - val reads = collect(join.right) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head assert(read.isLocalRead) @@ -865,17 +866,17 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL assert(bhj.length == 1) // Build side should do local read. - val buildSide = find(bhj.head.left)(_.isInstanceOf[ColumnarAQEShuffleReadExec]) + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) assert(buildSide.isDefined) - assert(buildSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].isLocalRead) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) - val probeSide = find(bhj.head.right)(_.isInstanceOf[ColumnarAQEShuffleReadExec]) + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) if (probeSideLocalRead || probeSideCoalescedRead) { assert(probeSide.isDefined) if (probeSideLocalRead) { - assert(probeSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].isLocalRead) + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) } else { - assert(probeSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].hasCoalescedPartition) + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) } } else { assert(probeSide.isEmpty) @@ -893,7 +894,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val smj = findTopLevelSortMergeJoin(plan) assert(smj.length == 1) assert(smj.head.isSkewJoin == optimizeSkewJoin) - val aqeReads = collect(smj.head) { case c: ColumnarAQEShuffleReadExec => c } + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } if (coalescedRead || optimizeSkewJoin) { assert(aqeReads.length == 2) if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) @@ -1067,7 +1068,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") assert( collect(adaptive) { - case c @ ColumnarAQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => assert(c.hasCoalescedPartition) c }.length == 1 @@ -1176,12 +1177,12 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL repartition => val query = s"SELECT /*+ $repartition */ * FROM testData" val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) - collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } match { + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { case Seq(aqeShuffleRead) => assert(aqeShuffleRead.partitionSpecs.size === 1) assert(!aqeShuffleRead.isLocalRead) case _ => - fail("There should be a ColumnarAQEShuffleReadExec") + fail("There should be a AQEShuffleReadExec") } } } @@ -1191,7 +1192,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { val query = "SELECT /*+ REPARTITION */ * FROM testData" val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) - collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } match { + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { case Seq(aqeShuffleRead) => assert(aqeShuffleRead.partitionSpecs.size === 4) assert(aqeShuffleRead.isLocalRead) @@ -1222,7 +1223,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL skewedPartitionNumber: Int, totalNumber: Int): Unit = { val (_, adaptive) = runAdaptiveAndVerifyResult(query) - val read = collect(adaptive) { case read: ColumnarAQEShuffleReadExec => read } + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } assert(read.size == 1) assert( read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == @@ -1259,7 +1260,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL .createOrReplaceTempView("t2") val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") - val aqeReads = collect(adaptive) { case c: ColumnarAQEShuffleReadExec => c } + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } assert(aqeReads.length == 2) aqeReads.foreach { c => @@ -1276,7 +1277,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") val coalesceRead = collect(adaptive) { - case r: ColumnarAQEShuffleReadExec if r.hasCoalescedPartition => r + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r } assert(coalesceRead.length == 1) // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala index 4bb8e964553e..de6abb0f52e0 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala @@ -16,6 +16,30 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} -class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait { + private val actions: Seq[DataFrame => DataFrame] = Seq( + "_1 = 500", + "_1 = 500 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1000 or _1 = 1500", + "_1 >= 500 and _1 < 1000", + "(_1 >= 500 and _1 < 1000) or (_1 >= 1500 and _1 < 1600)" + ).map(f => (df: DataFrame) => df.filter(f)) + + test("Gluten: test reading unaligned pages - test all types except timestamp") { + val df = spark + .range(0, 2000) + .selectExpr( + "id as _1", + "cast(id as short) as _3", + "cast(id as int) as _4", + "cast(id as float) as _5", + "cast(id as double) as _6", + "cast(id as decimal(20,0)) as _7", + "cast(cast(1618161925000 + id * 1000 * 60 * 60 * 24 as timestamp) as date) as _9" + ) + checkUnalignedPages(df)(actions: _*) + } +} 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 a186fcb7a72f..1c0f6ae1a142 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -28,7 +28,8 @@ 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 +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.util.Utils @@ -41,7 +42,7 @@ import org.apache.parquet.filter2.predicate.Operators.{Column => _, _} import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetOutputFormat} import org.apache.parquet.hadoop.util.HadoopInputFile -import java.sql.Date +import java.sql.{Date, Timestamp} import java.time.LocalDate import scala.reflect.ClassTag @@ -67,6 +68,44 @@ abstract class GltuenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) } + test(GlutenTestConstants.GLUTEN_TEST + "filter pushdown - timestamp") { + Seq(true, false).foreach { + java8Api => + Seq(CORRECTED, LEGACY).foreach { + rebaseMode => + val millisData = Seq( + "1000-06-14 08:28:53.123", + "1582-06-15 08:28:53.001", + "1900-06-16 08:28:53.0", + "2018-06-17 08:28:53.999") + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString + ) { + import testImplicits._ + withTempPath { + file => + millisData + .map(i => Tuple1(Timestamp.valueOf(i))) + .toDF + .write + .format(dataSourceName) + .save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { + df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } + } + } + } + } + } + } + test( GlutenTestConstants.GLUTEN_TEST + "Filter applied on merged Parquet schema with new column should work") { @@ -249,14 +288,8 @@ abstract class GltuenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ """.stripMargin) withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - val e = intercept[SparkException] { - sql(s"select a from $tableName where b > 0").collect() - } - assert( - e.getCause.isInstanceOf[RuntimeException] && e.getCause.getMessage.contains( - """Found duplicate field(s) b in read lowercase mode""")) + checkAnswer(sql(s"select a from $tableName where b > 0"), (1 until count).map(Row(_))) } - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { checkAnswer(sql(s"select A from $tableName where B > 0"), (1 until count).map(Row(_))) } diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala index 594353152fd3..ad1ae40f928c 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql._ -import org.apache.spark.sql.types.{DataTypes, StructField, StructType} /** A test suite that tests basic Parquet I/O. */ class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { @@ -28,20 +27,4 @@ class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { override protected def readResourceParquetFile(name: String): DataFrame = { spark.read.parquet(testFile(name)) } - - test( - GlutenTestConstants.GLUTEN_TEST + - "SPARK-35640: int as long should throw schema incompatible error") { - val data = (1 to 4).map(i => Tuple1(i)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) - - withParquetFile(data) { - path => - val errMsg = - intercept[Exception](spark.read.schema(readSchema).parquet(path).collect()).getMessage - assert( - errMsg.contains( - "BaseVector::compatibleKind( childOutputType->kind(), childRequestedType->kind())")) - } - } } 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 dfbeb746b793..2bd75448498a 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 @@ -53,6 +53,7 @@ class TestFileSourceScanExecTransformer( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) diff --git a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala index d82e84d50d96..aafb27cef469 100644 --- a/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala +++ b/gluten-ut/spark32/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala @@ -166,4 +166,23 @@ class GlutenHiveSQLQuerySuite extends GlutenSQLTestsTrait { ignoreIfNotExists = true, purge = false) } + + test("avoid unnecessary filter binding for subfield during scan") { + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + sql("DROP TABLE IF EXISTS test_subfield") + sql( + "CREATE TABLE test_subfield (name STRING, favorite_color STRING, " + + " label STRUCT) USING hive OPTIONS(fileFormat 'parquet')") + sql( + "INSERT INTO test_subfield VALUES('test_1', 'red', named_struct('label_1', 'label-a'," + + " 'label_2', 'label-b'))"); + val df = spark.sql("select * from test_subfield where name='test_1'") + checkAnswer(df, Seq(Row("test_1", "red", Row("label-a", "label-b")))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_subfield"), + ignoreIfNotExists = true, + purge = false) + } } diff --git a/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala index c92ad93ac496..d40a9735e4cc 100644 --- a/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala @@ -613,6 +613,7 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-36924: Cast IntegralType to DayTimeIntervalType") .exclude("SPARK-36924: Cast YearMonthIntervalType to IntegralType") .exclude("SPARK-36924: Cast IntegralType to YearMonthIntervalType") + .exclude("Cast should output null for invalid strings when ANSI is not enabled.") enableSuite[GlutenCastSuiteWithAnsiModeOn] .exclude("null cast") .exclude("cast string to date") @@ -720,6 +721,7 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("WeekDay") .exclude("WeekOfYear") .exclude("DateFormat") + .exclude("Gluten - DateFormat") .exclude("Hour") .exclude("Minute") .exclude("date add interval") @@ -1549,7 +1551,6 @@ class ClickHouseTestSettings extends BackendTestSettings { .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") .exclude("SPARK-35640: int as long should throw schema incompatible error") .exclude("SPARK-36726: test incorrect Parquet row group file offset") - .exclude("Gluten - SPARK-35640: int as long should throw schema incompatible error") enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") enableSuite[GlutenParquetRebaseDatetimeV1Suite] @@ -2012,6 +2013,9 @@ class ClickHouseTestSettings extends BackendTestSettings { "SELECT structFieldSimple.key, arrayFieldSimple[1] FROM tableWithSchema a where int_Field=1") .exclude("SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema") enableSuite[SparkFunctionStatistics] - + enableSuite[GlutenImplicitsTest] + .exclude("fallbackSummary with shuffle") + .exclude("fallbackSummary with cache") + .exclude("fallbackSummary with cached data and shuffle") } // scalastyle:on line.size.limit diff --git a/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala index 9662c4dd3d87..531d873dfe0a 100644 --- a/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala +++ b/gluten-ut/spark33/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala @@ -146,6 +146,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("to_unix_timestamp") // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") enableSuite[GlutenDecimalExpressionSuite] enableSuite[GlutenHashExpressionsSuite] enableSuite[GlutenIntervalExpressionsSuite] @@ -616,6 +618,8 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] enableSuite[GlutenParquetDeltaEncodingInteger] @@ -636,8 +640,11 @@ class VeloxTestSettings extends BackendTestSettings { // 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. + .exclude("filter pushdown - timestamp") .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") @@ -651,8 +658,11 @@ class VeloxTestSettings extends BackendTestSettings { // 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. + .exclude("filter pushdown - timestamp") .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") // Ignore Spark's filter pushdown check. .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") @@ -679,7 +689,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-34817: Read UINT_8/UINT_16/UINT_32 from parquet") // Exception. .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") - // Rewrite to align exception msg. + // Exception msg. .exclude("SPARK-35640: int as long should throw schema incompatible error") // Timestamp is read as INT96. .exclude("read dictionary and plain encoded timestamp_millis written as INT64") @@ -878,9 +888,7 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("Gluten - determining the number of reducers: join operator") .exclude("Gluten - determining the number of reducers: complex query 1") .exclude("Gluten - determining the number of reducers: complex query 2") - .exclude("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") .exclude("Union two datasets with different pre-shuffle partition number") - .exclude("SPARK-34790: enable IO encryption in AQE partition coalescing") enableSuite[GlutenExchangeSuite] // ColumnarShuffleExchangeExec does not support doExecute() method .exclude("shuffling UnsafeRows in exchange") @@ -1019,6 +1027,12 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenDataFrameTimeWindowingSuite] enableSuite[GlutenDataFrameTungstenSuite] enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .exclude("unbounded preceding/following range between with aggregation") + .exclude("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .exclude("rows between should accept int/long values as boundary") enableSuite[GlutenDataFrameWriterV2Suite] enableSuite[GlutenDatasetAggregatorSuite] enableSuite[GlutenDatasetCacheSuite] @@ -1043,10 +1057,14 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOnDisableProject] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOffDisableProject] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOnDisableProject] + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOffDisableProject] enableSuite[GlutenExpressionsSchemaSuite] enableSuite[GlutenExtraStrategiesSuite] enableSuite[GlutenFileBasedDataSourceSuite] @@ -1068,6 +1086,13 @@ class VeloxTestSettings extends BackendTestSettings { .exclude("SPARK-31116: Select nested schema with case insensitive mode") // exclude as original metric not correct when task offloaded to velox .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + .exclude("SPARK-23271 empty RDD when saved should write a metadata only file - orc") + .exclude("SPARK-22146 read files containing special characters using orc") + .exclude("SPARK-30362: test input metrics for DSV2") + .exclude("Do not use cache on overwrite") + .exclude("Do not use cache on append") + .exclude("File source v2: support partition pruning") + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") enableSuite[GlutenFileScanSuite] enableSuite[GlutenGeneratorFunctionSuite] enableSuite[GlutenInjectRuntimeFilterSuite] @@ -1128,5 +1153,6 @@ class VeloxTestSettings extends BackendTestSettings { enableSuite[GlutenXPathFunctionsSuite] enableSuite[GlutenFallbackSuite] enableSuite[GlutenHiveSQLQuerySuite] + enableSuite[GlutenImplicitsTest] } // scalastyle:on line.size.limit diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala index 07ab780c4659..744271e53a78 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql +import io.glutenproject.GlutenConfig + import org.apache.spark.sql.internal.SQLConf class GlutenBloomFilterAggregateQuerySuite @@ -25,9 +27,12 @@ class GlutenBloomFilterAggregateQuerySuite test("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") { val table = "bloom_filter_test" - withSQLConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS.key -> "5000000") { + withSQLConf( + SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS.key -> "5000000", + GlutenConfig.COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS.key -> "4194304" + ) { val numEstimatedItems = 5000000L - val numBits = SQLConf.get.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS) + val numBits = GlutenConfig.getConf.veloxBloomFilterMaxNumBits val sqlString = s""" |SELECT every(might_contain( | (SELECT bloom_filter_agg(col, diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala index 152b3ed66fc4..d6a04713bcd9 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql +import io.glutenproject.GlutenConfig + import org.apache.spark.SparkConf import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.columnar.InMemoryRelation @@ -24,9 +26,11 @@ class GlutenCachedTableSuite extends CachedTableSuite with GlutenSQLTestsTrait with AdaptiveSparkPlanHelper { - + // for temporarily disable the columnar table cache globally. + sys.props.put(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") override def sparkConf: SparkConf = { super.sparkConf.set("spark.sql.shuffle.partitions", "5") + super.sparkConf.set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") } test("GLUTEN - InMemoryRelation statistics") { diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala index e8a424de5be1..d0d856b5167b 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala @@ -16,4 +16,11 @@ */ package org.apache.spark.sql -class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait {} +class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Cancelling stage in a query with Range", + "SPARK-20430 Initialize Range parameters in a driver side" + ) + +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala index 9048c3845b35..3815a1369e07 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala @@ -726,6 +726,22 @@ class GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan } } +class GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOnDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOffWSCGOffDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + } +} + // Test DPP with batch scan disabled by user for some reason, which can also mock the situation // that scan is not transformable. class GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan @@ -742,3 +758,19 @@ class GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") } } + +class GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOnDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + } +} + +class GlutenDynamicPartitionPruningV2SuiteAEOffWSCGOffDisableProject + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf + .set(GlutenConfig.COLUMNAR_PROJECT_ENABLED.key, "false") + .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenImplicitsTest.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenImplicitsTest.scala new file mode 100644 index 000000000000..63348295feab --- /dev/null +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenImplicitsTest.scala @@ -0,0 +1,129 @@ +/* + * 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.SparkConf +import org.apache.spark.sql.execution.GlutenImplicits._ +import org.apache.spark.sql.internal.SQLConf + +class GlutenImplicitsTest extends GlutenSQLTestsBaseTrait { + sys.props.put("spark.gluten.sql.columnar.tableCache", "true") + + override protected def beforeAll(): Unit = { + super.beforeAll() + spark + .range(10) + .selectExpr("id as c1", "id % 3 as c2") + .write + .format("parquet") + .saveAsTable("t1") + } + + override protected def afterAll(): Unit = { + spark.sql("drop table t1") + super.afterAll() + } + + override protected def afterEach(): Unit = { + spark.catalog.clearCache() + super.afterEach() + } + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } + + private def withAQEEnabledAndDisabled(f: => Unit): Unit = { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "true", + SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "true" + ) { + f + } + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.ADAPTIVE_EXECUTION_FORCE_APPLY.key -> "false", + SQLConf.CAN_CHANGE_CACHED_PLAN_OUTPUT_PARTITIONING.key -> "false" + ) { + f + } + } + + test("fallbackSummary with query") { + withAQEEnabledAndDisabled { + val df = spark.table("t1").filter(_.getLong(0) > 0) + assert(df.fallbackSummary().numGlutenNodes == 1, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + df.collect() + assert(df.fallbackSummary().numGlutenNodes == 1, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + } + } + + test("fallbackSummary with shuffle") { + withAQEEnabledAndDisabled { + val df = spark.sql("SELECT c2 FROM t1 group by c2").filter(_.getLong(0) > 0) + assert(df.fallbackSummary().numGlutenNodes == 5, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + df.collect() + assert(df.fallbackSummary().numGlutenNodes == 5, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + } + } + + test("fallbackSummary with set command") { + withAQEEnabledAndDisabled { + val df = spark.sql("set k=v") + assert(df.fallbackSummary().numGlutenNodes == 0, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 0, df.fallbackSummary()) + } + } + + test("fallbackSummary with data write command") { + withAQEEnabledAndDisabled { + withTable("tmp") { + val df = spark.sql("create table tmp using parquet as select * from t1") + assert(df.fallbackSummary().numGlutenNodes == 1, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 0, df.fallbackSummary()) + } + } + } + + test("fallbackSummary with cache") { + withAQEEnabledAndDisabled { + val df = spark.table("t1").cache().filter(_.getLong(0) > 0) + assert(df.fallbackSummary().numGlutenNodes == 2, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + df.collect() + assert(df.fallbackSummary().numGlutenNodes == 2, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + } + } + + test("fallbackSummary with cached data and shuffle") { + withAQEEnabledAndDisabled { + val df = spark.sql("select * from t1").filter(_.getLong(0) > 0).cache.repartition() + assert(df.fallbackSummary().numGlutenNodes == 3, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + df.collect() + assert(df.fallbackSummary().numGlutenNodes == 3, df.fallbackSummary()) + assert(df.fallbackSummary().numFallbackNodes == 1, df.fallbackSummary()) + } + } +} diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala index 891dd473f2a2..90046c0b15b2 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -212,7 +212,9 @@ class GlutenSQLQueryTestSuite "explain.sql", // explain "group-analytics.sql", // wait velox to fix issue 3357 "array.sql", // blocked by VELOX-5768 - "higher-order-functions.sql" // blocked by VELOX-5768 + "higher-order-functions.sql", // blocked by VELOX-5768 + "udf/udf-window.sql", // Local window fixes are not added. + "window.sql" // Local window fixes are not added. ) ++ otherIgnoreList /** @@ -240,8 +242,9 @@ class GlutenSQLQueryTestSuite "current_database_catalog.sql", "date.sql", "datetime-formatting-invalid.sql", - "datetime-formatting-legacy.sql", - "datetime-formatting.sql", + // Velox had different handling for some illegal cases. +// "datetime-formatting-legacy.sql", +// "datetime-formatting.sql", "datetime-legacy.sql", "datetime-parsing-invalid.sql", "datetime-parsing-legacy.sql", diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala index 5d24d7e20439..b599a277214b 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala @@ -285,4 +285,62 @@ class GlutenDateExpressionsSuite extends DateExpressionsSuite with GlutenTestsTr GenerateUnsafeProjection.generate( ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) } + + // Modified based on vanilla spark to explicitly set timezone in config. + test(GLUTEN_TEST + "DateFormat") { + val PST_OPT = Option(PST.getId) + val JST_OPT = Option(JST.getId) + + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), + null) + checkEvaluation( + DateFormatClass( + Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), + UTC_OPT), + null) + + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("y"), UTC_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("H"), UTC_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("y"), PST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("H"), PST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("y"), JST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("H"), JST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + } + } + } } diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala index f2b7966e2c46..bc80c1b8f3e3 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala @@ -19,23 +19,16 @@ package org.apache.spark.sql.execution import org.apache.spark.SparkConf import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED import org.apache.spark.internal.config.UI.UI_ENABLED -import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, Row, SparkSession} +import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, SparkSession} import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST -import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ColumnarAQEShuffleReadExec, QueryStageExec, ShuffleQueryStageExec} -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec -import org.apache.spark.sql.functions.{col, max} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf class GlutenCoalesceShufflePartitionsSuite extends CoalesceShufflePartitionsSuite with GlutenTestsCommonTrait { - object ColumnarCoalescedShuffleRead { - def unapply(read: ColumnarAQEShuffleReadExec): Boolean = { - !read.isLocalRead && !read.hasSkewedPartition && read.hasCoalescedPartition - } - } - override protected def afterAll(): Unit = {} override def withSparkSession( @@ -78,111 +71,6 @@ class GlutenCoalesceShufflePartitionsSuite } } - test( - GLUTEN_TEST + - "SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") { - val test: SparkSession => Unit = { - spark: SparkSession => - spark.sql("SET spark.sql.exchange.reuse=true") - val df = spark.range(0, 6, 1).selectExpr("id AS key", "id AS value") - - // test case 1: a query stage has 3 child stages but they are the same stage. - // Final Stage 1 - // ShuffleQueryStage 0 - // ReusedQueryStage 0 - // ReusedQueryStage 0 - val resultDf = df.join(df, "key").join(df, "key") - QueryTest.checkAnswer(resultDf, (0 to 5).map(i => Row(i, i, i, i))) - val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec] - .executedPlan - assert(finalPlan.collect { - case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r - }.length == 2) - assert(finalPlan.collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - }.length == 3) - - // test case 2: a query stage has 2 parent stages. - // Final Stage 3 - // ShuffleQueryStage 1 - // ShuffleQueryStage 0 - // ShuffleQueryStage 2 - // ReusedQueryStage 0 - val grouped = df.groupBy("key").agg(max("value").as("value")) - val resultDf2 = grouped - .groupBy(col("key") + 1) - .max("value") - .union(grouped.groupBy(col("key") + 2).max("value")) - QueryTest.checkAnswer( - resultDf2, - Row(1, 0) :: Row(2, 0) :: Row(2, 1) :: Row(3, 1) :: - Row(3, 2) :: Row(4, 2) :: Row(4, 3) :: Row(5, 3) :: Row(5, 4) :: Row(6, 4) :: Row( - 6, - 5) :: - Row(7, 5) :: Nil) - - val finalPlan2 = resultDf2.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec] - .executedPlan - - // The result stage has 2 children - val level1Stages = finalPlan2.collect { case q: QueryStageExec => q } - assert(level1Stages.length == 2) - level1Stages.foreach( - qs => - assert( - qs.plan.collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - }.length == 1, - "Wrong CoalescedShuffleRead below " + qs.simpleString(3) - )) - - val leafStages = level1Stages.flatMap { - stage => - // All of the child stages of result stage have only one child stage. - val children = stage.plan.collect { case q: QueryStageExec => q } - assert(children.length == 1) - children - } - assert(leafStages.length == 2) - - val reusedStages = level1Stages.flatMap { - stage => - stage.plan.collect { case ShuffleQueryStageExec(_, r: ReusedExchangeExec, _) => r } - } - assert(reusedStages.length == 1) - } - withSparkSession(test, 400, None) - } - - test(GLUTEN_TEST + "SPARK-34790: enable IO encryption in AQE partition coalescing") { - val test: SparkSession => Unit = { - spark: SparkSession => - val ds = spark.range(0, 100, 1, numInputPartitions) - val resultDf = ds.repartition(ds.col("id")) - resultDf.collect() - - val finalPlan = resultDf.queryExecution.executedPlan - .asInstanceOf[AdaptiveSparkPlanExec] - .executedPlan - assert( - finalPlan - .collect { - case r @ CoalescedShuffleRead() => r - case c @ ColumnarCoalescedShuffleRead() => c - } - .isDefinedAt(0)) - } - Seq(true, false).foreach { - enableIOEncryption => - // Before SPARK-34790, it will throw an exception when io encryption enabled. - withSparkSession(test, Int.MaxValue, None, enableIOEncryption) - } - } - Seq(Some(5), None).foreach { minNumPostShufflePartitions => val testNameNote = minNumPostShufflePartitions match { @@ -208,11 +96,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = agg.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -255,11 +139,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -308,11 +188,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -362,11 +238,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } minNumPostShufflePartitions match { case Some(numPartitions) => @@ -411,11 +283,7 @@ class GlutenCoalesceShufflePartitionsSuite val finalPlan = join.queryExecution.executedPlan .asInstanceOf[AdaptiveSparkPlanExec] .executedPlan - val shuffleReads = finalPlan.collect { - case r @ CoalescedShuffleRead() => r - // Added for gluten. - case r @ ColumnarCoalescedShuffleRead() => r - } + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } assert(shuffleReads.length === 0) } finally { spark.sql("drop table t") diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala index 9a9dd77e4364..5b5ee83be49b 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala @@ -146,7 +146,6 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length val numLocalReads = collect(plan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read case r: AQEShuffleReadExec if r.isLocalRead => r } // because columnar local reads cannot execute @@ -207,7 +206,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL assert(sortMergeJoinSize(plan) == 1) assert(broadcastHashJoinSize(adaptivePlan) == 1) val localReads = collect(adaptivePlan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read + case read: AQEShuffleReadExec if read.isLocalRead => read } assert(localReads.length == 2) } @@ -225,14 +224,14 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) assert(bhj.size == 1) val localReads = collect(adaptivePlan) { - case read: ColumnarAQEShuffleReadExec if read.isLocalRead => read + case read: AQEShuffleReadExec if read.isLocalRead => read } assert(localReads.length == 2) val localShuffleRDD0 = localReads(0) - .doExecuteColumnar() + .executeColumnar() .asInstanceOf[ShuffledColumnarBatchRDD] val localShuffleRDD1 = localReads(1) - .doExecuteColumnar() + .executeColumnar() .asInstanceOf[ShuffledColumnarBatchRDD] // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 // and the partitions length is 2 * numMappers = 4 @@ -271,7 +270,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL checkAnswer(testDf, Seq()) val plan = testDf.queryExecution.executedPlan assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformer]).isDefined) - val coalescedReads = collect(plan) { case r: ColumnarAQEShuffleReadExec => r } + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } assert(coalescedReads.length == 3, s"$plan") coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) } @@ -438,8 +437,10 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL test("gluten Exchange reuse") { withSQLConf( SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "100", - SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + // magic threshold, ch backend has two bhj when threshold is 100 + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "90", + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( "SELECT value FROM testData join testData2 ON key = a " + "join (SELECT value v from testData join testData3 ON key = a) on value = v") @@ -673,7 +674,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL } assert( joins.head.left - .collect { case r: ColumnarAQEShuffleReadExec => r } + .collect { case r: AQEShuffleReadExec => r } .head .partitionSpecs .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } @@ -681,7 +682,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL .length == leftSkewNum) assert( joins.head.right - .collect { case r: ColumnarAQEShuffleReadExec => r } + .collect { case r: AQEShuffleReadExec => r } .head .partitionSpecs .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } @@ -717,10 +718,10 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL test("gluten SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") - val reads = collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head - val c = read.canonicalized.asInstanceOf[ColumnarAQEShuffleReadExec] + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] // we can't just call execute() because that has separate checks for canonicalized plans val ex = intercept[IllegalStateException] { val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) @@ -735,7 +736,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.SHUFFLE_PARTITIONS.key -> "5") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") - val reads = collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head assert(!read.isLocalRead) @@ -756,7 +757,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformer => j }.head assert(join.joinBuildSide == BuildLeft) - val reads = collect(join.right) { case r: ColumnarAQEShuffleReadExec => r } + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } assert(reads.length == 1) val read = reads.head assert(read.isLocalRead) @@ -867,17 +868,17 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL assert(bhj.length == 1) // Build side should do local read. - val buildSide = find(bhj.head.left)(_.isInstanceOf[ColumnarAQEShuffleReadExec]) + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) assert(buildSide.isDefined) - assert(buildSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].isLocalRead) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) - val probeSide = find(bhj.head.right)(_.isInstanceOf[ColumnarAQEShuffleReadExec]) + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) if (probeSideLocalRead || probeSideCoalescedRead) { assert(probeSide.isDefined) if (probeSideLocalRead) { - assert(probeSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].isLocalRead) + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) } else { - assert(probeSide.get.asInstanceOf[ColumnarAQEShuffleReadExec].hasCoalescedPartition) + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) } } else { assert(probeSide.isEmpty) @@ -895,7 +896,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val smj = findTopLevelSortMergeJoin(plan) assert(smj.length == 1) assert(smj.head.isSkewJoin == optimizeSkewJoin) - val aqeReads = collect(smj.head) { case c: ColumnarAQEShuffleReadExec => c } + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } if (coalescedRead || optimizeSkewJoin) { assert(aqeReads.length == 2) if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) @@ -1071,7 +1072,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") assert( collect(adaptive) { - case c @ ColumnarAQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => assert(c.hasCoalescedPartition) c }.length == 1 @@ -1180,12 +1181,12 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL repartition => val query = s"SELECT /*+ $repartition */ * FROM testData" val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) - collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } match { + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { case Seq(aqeShuffleRead) => assert(aqeShuffleRead.partitionSpecs.size === 1) assert(!aqeShuffleRead.isLocalRead) case _ => - fail("There should be a ColumnarAQEShuffleReadExec") + fail("There should be a AQEShuffleReadExec") } } } @@ -1195,7 +1196,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { val query = "SELECT /*+ REPARTITION */ * FROM testData" val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) - collect(adaptivePlan) { case r: ColumnarAQEShuffleReadExec => r } match { + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { case Seq(aqeShuffleRead) => assert(aqeShuffleRead.partitionSpecs.size === 4) assert(aqeShuffleRead.isLocalRead) @@ -1226,7 +1227,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL skewedPartitionNumber: Int, totalNumber: Int): Unit = { val (_, adaptive) = runAdaptiveAndVerifyResult(query) - val read = collect(adaptive) { case read: ColumnarAQEShuffleReadExec => read } + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } assert(read.size == 1) assert( read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == @@ -1263,7 +1264,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL .createOrReplaceTempView("t2") val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") - val aqeReads = collect(adaptive) { case c: ColumnarAQEShuffleReadExec => c } + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } assert(aqeReads.length == 2) aqeReads.foreach { c => @@ -1280,7 +1281,7 @@ class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQL val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") val coalesceRead = collect(adaptive) { - case r: ColumnarAQEShuffleReadExec if r.hasCoalescedPartition => r + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r } assert(coalesceRead.length == 1) // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala index 4bb8e964553e..e69577fc621a 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala @@ -16,6 +16,30 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} -class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait {} +class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait { + private val actions: Seq[DataFrame => DataFrame] = Seq( + "_1 = 500", + "_1 = 500 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1000 or _1 = 1500", + "_1 >= 500 and _1 < 1000", + "(_1 >= 500 and _1 < 1000) or (_1 >= 1500 and _1 < 1600)" + ).map(f => (df: DataFrame) => df.filter(f)) + + test("Gluten: test reading unaligned pages - test all types") { + val df = spark + .range(0, 2000) + .selectExpr( + "id as _1", + "cast(id as short) as _3", + "cast(id as int) as _4", + "cast(id as float) as _5", + "cast(id as double) as _6", + "cast(id as decimal(20,0)) as _7", + "cast(cast(1618161925000 + id * 1000 * 60 * 60 * 24 as timestamp) as date) as _9" + ) + checkUnalignedPages(df)(actions: _*) + } +} 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 ebf75d146dc6..2e46d6eb477e 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -28,7 +28,8 @@ 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 +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, LEGACY} +import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType.INT96 import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.util.Utils @@ -41,7 +42,7 @@ import org.apache.parquet.filter2.predicate.Operators.{Column => _, _} import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetOutputFormat} import org.apache.parquet.hadoop.util.HadoopInputFile -import java.sql.Date +import java.sql.{Date, Timestamp} import java.time.LocalDate import scala.reflect.ClassTag @@ -66,9 +67,45 @@ abstract class GltuenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ spark.read.parquet( getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) } - test( - GlutenTestConstants.GLUTEN_TEST + - "SPARK-40280: filter pushdown - int with annotation") {} + + test(GlutenTestConstants.GLUTEN_TEST + "filter pushdown - timestamp") { + Seq(true, false).foreach { + java8Api => + Seq(CORRECTED, LEGACY).foreach { + rebaseMode => + val millisData = Seq( + "1000-06-14 08:28:53.123", + "1582-06-15 08:28:53.001", + "1900-06-16 08:28:53.0", + "2018-06-17 08:28:53.999") + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString + ) { + import testImplicits._ + withTempPath { + file => + millisData + .map(i => Tuple1(Timestamp.valueOf(i))) + .toDF + .write + .format(dataSourceName) + .save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { + df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } + } + } + } + } + } + } + test( GlutenTestConstants.GLUTEN_TEST + "Filter applied on merged Parquet schema with new column should work") { @@ -229,43 +266,6 @@ abstract class GltuenParquetFilterSuite extends ParquetFilterSuite with GlutenSQ } } - test( - GlutenTestConstants.GLUTEN_TEST + - "SPARK-25207: exception when duplicate fields in case-insensitive mode") { - withTempPath { - dir => - val count = 10 - val tableName = "spark_25207" - val tableDir = dir.getAbsoluteFile + "/table" - withTable(tableName) { - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - spark - .range(count) - .selectExpr("id as A", "id as B", "id as b") - .write - .mode("overwrite") - .parquet(tableDir) - } - sql(s""" - |CREATE TABLE $tableName (A LONG, B LONG) USING PARQUET LOCATION '$tableDir' - """.stripMargin) - - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - val e = intercept[SparkException] { - sql(s"select a from $tableName where b > 0").collect() - } - assert( - e.getCause.isInstanceOf[RuntimeException] && e.getCause.getMessage.contains( - """Found duplicate field(s) b in read lowercase mode""")) - } - - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - checkAnswer(sql(s"select A from $tableName where B > 0"), (1 until count).map(Row(_))) - } - } - } - } - test(GlutenTestConstants.GLUTEN_TEST + "Support Parquet column index") { // block 1: // null count min max diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala index b65d6556a50a..09bc0fc39ee1 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.datasources.parquet import org.apache.spark.sql._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{DataTypes, StructField, StructType} /** A test suite that tests basic Parquet I/O. */ class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { @@ -41,20 +40,4 @@ class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { override protected def readResourceParquetFile(name: String): DataFrame = { spark.read.parquet(testFile(name)) } - - test( - GlutenTestConstants.GLUTEN_TEST + - "SPARK-35640: int as long should throw schema incompatible error") { - val data = (1 to 4).map(i => Tuple1(i)) - val readSchema = StructType(Seq(StructField("_1", DataTypes.LongType))) - - withParquetFile(data) { - path => - val errMsg = - intercept[Exception](spark.read.schema(readSchema).parquet(path).collect()).getMessage - assert( - errMsg.contains( - "BaseVector::compatibleKind( childOutputType->kind(), childRequestedType->kind())")) - } - } } 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 2b4670512693..36fe19bd7039 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 @@ -52,6 +52,7 @@ class TestFileSourceScanExecTransformer( relation, selectedPartitions, output, + bucketedScan, optionalBucketSet, optionalNumCoalescedBuckets, disableBucketedScan) 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 62b095482826..8a2acc9db1e6 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 @@ -74,7 +74,7 @@ class GlutenFallbackSuite extends GlutenSQLTestsTrait { val id = runExecution("SELECT * FROM t") val execution = glutenStore.execution(id) assert(execution.isDefined) - assert(execution.get.numGlutenNodes == 2) + assert(execution.get.numGlutenNodes == 1) assert(execution.get.numFallbackNodes == 0) assert(execution.get.fallbackNodeToReason.isEmpty) @@ -83,7 +83,7 @@ class GlutenFallbackSuite extends GlutenSQLTestsTrait { val execution = glutenStore.execution(id) assert(execution.isDefined) assert(execution.get.numGlutenNodes == 0) - assert(execution.get.numFallbackNodes == 2) + assert(execution.get.numFallbackNodes == 1) val fallbackReason = execution.get.fallbackNodeToReason.head assert(fallbackReason._1.contains("Scan parquet default.t")) assert(fallbackReason._2.contains("columnar FileScan is not enabled in FileSourceScanExec")) diff --git a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala index d82e84d50d96..90d978b0d2e5 100644 --- a/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala +++ b/gluten-ut/spark33/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala @@ -166,4 +166,23 @@ class GlutenHiveSQLQuerySuite extends GlutenSQLTestsTrait { ignoreIfNotExists = true, purge = false) } + + test("avoid unnecessary filter binding for subfield during scan") { + withSQLConf("spark.sql.hive.convertMetastoreParquet" -> "false") { + sql("DROP TABLE IF EXISTS test_subfield") + sql( + "CREATE TABLE test_subfield (name STRING, favorite_color STRING," + + " label STRUCT) USING hive OPTIONS(fileFormat 'parquet')") + sql( + "INSERT INTO test_subfield VALUES('test_1', 'red', named_struct('label_1', 'label-a'," + + "'label_2', 'label-b'))"); + val df = spark.sql("select * from test_subfield where name='test_1'") + checkAnswer(df, Seq(Row("test_1", "red", Row("label-a", "label-b")))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_subfield"), + ignoreIfNotExists = true, + purge = false) + } } diff --git a/gluten-ut/spark34/pom.xml b/gluten-ut/spark34/pom.xml new file mode 100644 index 000000000000..b7f20843387c --- /dev/null +++ b/gluten-ut/spark34/pom.xml @@ -0,0 +1,189 @@ + + + + gluten-ut + io.glutenproject + 1.1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + gluten-ut-spark34 + jar + Gluten Unit Test Spark34 + + + + io.glutenproject + gluten-ut-common + ${project.version} + compile + test-jar + + + org.apache.parquet + parquet-column + 1.12.3 + test + tests + + + + + + backends-clickhouse + + false + + + + io.glutenproject + backends-clickhouse + ${project.version} + test + + + org.apache.celeborn + celeborn-client-spark-${spark.major.version}-shaded_${scala.binary.version} + ${celeborn.version} + test + + + + + backends-velox + + false + + + + io.glutenproject + backends-velox + ${project.version} + test + + + org.apache.arrow + arrow-vector + ${arrow.version} + + + io.netty + netty-common + + + io.netty + netty-buffer + + + test + + + org.apache.arrow + arrow-c-data + ${arrow.version} + test + + + org.apache.arrow + arrow-vector + + + protobuf-java + com.google.protobuf + + + + + org.apache.arrow + arrow-memory-netty + ${arrow.version} + test + + + org.apache.arrow + arrow-memory-core + ${arrow.version} + test + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-resources-plugin + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.scalatest + scalatest-maven-plugin + + + test + + test + + + + ${clickhouse.lib.path} + ${tpcds.data.path} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + prepare-test-jar + test-compile + + test-jar + + + + + + + diff --git a/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala new file mode 100644 index 000000000000..733d301a7cb3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/clickhouse/ClickHouseTestSettings.scala @@ -0,0 +1,1800 @@ +/* + * 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 io.glutenproject.utils.clickhouse + +import io.glutenproject.utils.BackendTestSettings + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector._ +import org.apache.spark.sql.errors._ +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.GlutenAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.exchange._ +import org.apache.spark.sql.execution.datasources.json._ +import org.apache.spark.sql.execution.datasources.orc._ +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.GlutenEnsureRequirementsSuite +import org.apache.spark.sql.execution.joins.{GlutenBroadcastJoinSuite, GlutenExistenceJoinSuite, GlutenInnerJoinSuite, GlutenOuterJoinSuite} +import org.apache.spark.sql.extension.{GlutenCustomerExtensionSuite, GlutenSessionExtensionSuite} +import org.apache.spark.sql.gluten.GlutenFallbackSuite +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.statistics.SparkFunctionStatistics + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class ClickHouseTestSettings extends BackendTestSettings { + + // disable tests that will break the whole UT + override def shouldRun(suiteName: String, testName: String): Boolean = { + val preCheck = suiteName.split("[.]").last match { + case "GlutenCSVSuite" => !csvCoreDumpCases.contains(testName) + case "GlutenCSVv1Suite" => !csvCoreDumpCases.contains(testName) + case "GlutenCSVv2Suite" => !csvCoreDumpCases.contains(testName) + case "GlutenCSVLegacyTimeParserSuite" => !csvCoreDumpCases.contains(testName) + case "GlutenDataFrameSuite" => !dfCoreDumpCases.contains(testName) + case "GlutenDatasetSuite" => !dsSlowCases.contains(testName) + case "GlutenSQLQuerySuite" => !sqlQuerySlowCases.contains(testName) + case "GlutenDataFrameWriterV2Suite" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenDataSourceV2DataFrameSuite" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenDataSourceV2FunctionSuite" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenDataSourceV2SQLSuiteV1Filter" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenDataSourceV2SQLSuiteV2Filter" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenMetadataColumnSuite" => false // nativeDoValidate failed due to spark conf cleanup + case "GlutenQueryCompilationErrorsDSv2Suite" => + false // nativeDoValidate failed due to spark conf cleanup + case "GlutenBloomFilterAggregateQuerySuite" => + !bloomFilterCases.contains(testName) + case _ => true + } + preCheck && super.shouldRun(suiteName, testName) + } + + private val csvCoreDumpCases: Seq[String] = Seq( + "test with alternative delimiter and quote", + "SPARK-24540: test with multiple character delimiter (comma space)", + "DDL test with tab separated file", + "test with null quote character", + "SPARK-24540: test with multiple (crazy) character delimiter", + "nullable fields with user defined null value of \"null\"", + "SPARK-15585 turn off quotations", + "SPARK-29101 test count with DROPMALFORMED mode" + ) + + private val dfCoreDumpCases: Seq[String] = Seq( + "repartitionByRange", + "Gluten - repartitionByRange" + ) + + private val dsSlowCases: Seq[String] = Seq( + "SPARK-16995: flat mapping on Dataset containing a column created with lit/expr" + ) + + private val sqlQuerySlowCases: Seq[String] = Seq( + "SPARK-33084: Add jar support Ivy URI in SQL" + ) + + private val bloomFilterCases: Seq[String] = Seq( + // Currently return a empty set(same reason as sum(empty set), + // both behaviors are acceptable. + "Test that bloom_filter_agg produces a NULL with empty input" + ) + + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite].exclude( + "test ApproxCountDistinctForIntervals with large number of endpoints") + enableSuite[GlutenApproximatePercentileQuerySuite].exclude( + "SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenBloomFilterAggregateQuerySuite] + .exclude("Test bloom_filter_agg and might_contain") + .exclude("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") + enableSuite[GlutenCTEHintSuite] + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenCachedTableSuite] + .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + .exclude("analyzes column statistics in cached query") + .exclude("GLUTEN - InMemoryRelation statistics") + enableSuite[GlutenColumnExpressionSuite] + .exclude("input_file_name, input_file_block_start, input_file_block_length - FileScanRDD") + .exclude("withField should add field with no name") + .exclude("withField should add field to nullable struct") + .exclude("withField should add field to nested nullable struct") + .exclude("withField should add multiple fields to nullable struct") + .exclude("withField should add multiple fields to nested nullable struct") + .exclude("withField should replace field in nullable struct") + .exclude("withField should replace field in nested nullable struct") + .exclude("withField should replace multiple fields in nullable struct") + .exclude("withField should replace multiple fields in nested nullable struct") + .exclude("withField should replace all fields with given name in struct") + .exclude("withField user-facing examples") + .exclude("dropFields should drop field in nullable struct") + .exclude("dropFields should drop field with no name in struct") + .exclude("dropFields should drop field in nested nullable struct") + .exclude("dropFields should drop multiple fields in nested nullable struct") + .exclude("dropFields should drop all fields with given name in struct") + .exclude("dropFields user-facing examples") + .exclude("should move field up one level of nesting") + .exclude("SPARK-36778: add ilike API for scala") + enableSuite[GlutenComplexTypesSuite] + enableSuite[GlutenConfigBehaviorSuite].exclude( + "SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenDSV2SQLInsertTestSuite] + enableSuite[GlutenDataFrameAggregateSuite] + .exclude("average") + .exclude("zero average") + .exclude("zero stddev") + .exclude("collect functions") + .exclude("collect functions structs") + .exclude("SPARK-17641: collect functions should not collect null values") + .exclude("collect functions should be able to cast to array type with no null values") + .exclude("SPARK-17616: distinct aggregate combined with a non-partial aggregate") + .exclude("SPARK-19471: AggregationIterator does not initialize the generated result projection before using it") + .exclude("SPARK-26021: NaN and -0.0 in grouping expressions") + .exclude("SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") + .exclude("SPARK-32136: NormalizeFloatingNumbers should work on null struct") + .exclude("SPARK-34713: group by CreateStruct with ExtractValue") + .exclude("SPARK-34716: Support ANSI SQL intervals by the aggregate function `sum`") + .exclude("SPARK-34837: Support ANSI SQL intervals by the aggregate function `avg`") + .exclude("SPARK-35412: groupBy of year-month/day-time intervals should work") + .exclude("SPARK-36926: decimal average mistakenly overflow") + .exclude("Gluten - use gluten hash agg to replace vanilla spark sort agg") + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + .exclude("map with arrays") + .exclude("bin") + .exclude("sequence") + .exclude("element_at function") + .exclude("aggregate function - array for primitive type not containing null") + .exclude("aggregate function - array for primitive type containing null") + .exclude("aggregate function - array for non-primitive type") + .exclude("transform keys function - primitive data types") + .exclude("transform values function - test empty") + .exclude("SPARK-14393: values generated by non-deterministic functions shouldn't change after coalesce or union") + .exclude("SPARK-24734: Fix containsNull of Concat for array type") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite].exclude( + "SPARK-32693: Compare two dataframes with same schema except nullable property") + enableSuite[GlutenDataFrameNaFunctionsSuite] + .exclude("replace nan with float") + .exclude("replace nan with double") + enableSuite[GlutenDataFramePivotSuite] + .exclude("pivot with column definition in groupby") + .exclude("pivot with timestamp and count should not print internal representation") + .exclude("SPARK-38133: Grouping by TIMESTAMP_NTZ should not corrupt results") + enableSuite[GlutenDataFrameRangeSuite] + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + .exclude("simple session window with record at window start") + .exclude("session window groupBy statement") + .exclude("SPARK-36465: filter out events with negative/zero gap duration") + .exclude("session window groupBy with multiple keys statement") + .exclude("session window groupBy with multiple keys statement - one distinct") + .exclude("session window groupBy with multiple keys statement - two distinct") + .exclude("session window groupBy with multiple keys statement - keys overlapped with sessions") + .exclude("session window with multi-column projection") + .exclude("SPARK-36724: Support timestamp_ntz as a type of time column for SessionWindow") + enableSuite[GlutenDataFrameSetOperationsSuite] + .exclude("SPARK-10740: handle nondeterministic expressions correctly for set operations") + .exclude( + "SPARK-34283: SQL-style union using Dataset, keep necessary deduplicate in multiple unions") + .exclude("union should union DataFrames with UDTs (SPARK-13410)") + .exclude( + "SPARK-32376: Make unionByName null-filling behavior work with struct columns - simple") + .exclude( + "SPARK-32376: Make unionByName null-filling behavior work with struct columns - nested") + .exclude("SPARK-32376: Make unionByName null-filling behavior work with struct columns - case-sensitive cases") + .exclude( + "SPARK-32376: Make unionByName null-filling behavior work with struct columns - edge case") + .exclude("SPARK-35290: Make unionByName null-filling behavior work with struct columns - sorting edge case") + .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-36797: Union should resolve nested columns as top-level columns") + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + enableSuite[GlutenDataFrameStatSuite] + enableSuite[GlutenDataFrameSuite] + .exclude("Uuid expressions should produce same results at retries in the same DataFrame") + .exclude("SPARK-28224: Aggregate sum big decimal overflow") + .exclude("SPARK-28067: Aggregate sum should not return wrong results for decimal overflow") + .exclude("SPARK-35955: Aggregate avg should not return wrong results for decimal overflow") + .exclude("describe") + .exclude("SPARK-34165: Add count_distinct to summary") + .exclude("getRows: array") + .exclude("showString: array") + .exclude("showString: array, vertical = true") + .exclude("SPARK-23023 Cast rows to strings in showString") + .exclude("SPARK-18350 show with session local timezone") + .exclude("SPARK-18350 show with session local timezone, vertical = true") + .exclude("SPARK-6899: type should match when using codegen") + .exclude("SPARK-7324 dropDuplicates") + .exclude( + "SPARK-8608: call `show` on local DataFrame with random columns should return same value") + .exclude("SPARK-8609: local DataFrame with random columns should return same value after sort") + .exclude("SPARK-9083: sort with non-deterministic expressions") + .exclude("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") + .exclude("distributeBy and localSort") + .exclude("reuse exchange") + .exclude("SPARK-22271: mean overflows and returns null for some decimal variables") + .exclude("SPARK-22520: support code generation for large CaseWhen") + .exclude("SPARK-24165: CaseWhen/If - nullability of nested types") + .exclude("SPARK-27671: Fix analysis exception when casting null in nested field in struct") + .exclude("Gluten - distributeBy and localSort") + .exclude("Gluten - describe") + .exclude("Gluten - Allow leading/trailing whitespace in string before casting") + enableSuite[GlutenDataFrameTimeWindowingSuite] + .exclude("simple tumbling window with record at window start") + .exclude("SPARK-21590: tumbling window using negative start time") + .exclude("tumbling window groupBy statement") + .exclude("tumbling window groupBy statement with startTime") + .exclude("SPARK-21590: tumbling window groupBy statement with negative startTime") + .exclude("tumbling window with multi-column projection") + .exclude("sliding window grouping") + .exclude("time window joins") + .exclude("negative timestamps") + .exclude("millisecond precision sliding windows") + enableSuite[GlutenDataFrameTungstenSuite].exclude("Map type with struct type as key") + enableSuite[GlutenDataFrameWindowFramesSuite] + .exclude("rows between should accept int/long values as boundary") + .exclude("range between should accept int/long values as boundary") + .exclude("reverse preceding/following range between with aggregation") + enableSuite[GlutenDataFrameWindowFunctionsSuite] + .exclude("corr, covar_pop, stddev_pop functions in specific window") + .exclude( + "SPARK-13860: corr, covar_pop, stddev_pop functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .exclude("covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") + .exclude("SPARK-13860: covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window LEGACY_STATISTICAL_AGGREGATE off") + .exclude("lead/lag with ignoreNulls") + .exclude("Window spill with more than the inMemoryThreshold and spillThreshold") + .exclude("SPARK-21258: complex object in combination with spilling") + .exclude( + "SPARK-38237: require all cluster keys for child required distribution for window query") + .exclude("Gluten - corr, covar_pop, stddev_pop functions in specific window") + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + .exclude("SPARK-16853: select, case class and tuple") + .exclude("select 2, primitive and tuple") + .exclude("SPARK-15550 Dataset.show() should show inner nested products as rows") + .exclude("dropDuplicates") + .exclude("dropDuplicates: columns with same column name") + .exclude("SPARK-24762: select Option[Product] field") + .exclude("SPARK-24762: typed agg on Option[Product] type") + .exclude("SPARK-26233: serializer should enforce decimal precision and scale") + .exclude("groupBy.as") + .exclude("SPARK-40407: repartition should not result in severe data skew") + .exclude("SPARK-40660: Switch to XORShiftRandom to distribute elements") + enableSuite[GlutenDateFunctionsSuite] + .exclude("function to_date") + .exclude("function trunc") + .exclude("from_unixtime") + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + .exclude("to_timestamp") + .exclude("to_timestamp with microseconds precision") + .exclude("SPARK-30668: use legacy timestamp parser in to_timestamp") + .exclude("SPARK-30766: date_trunc of old timestamps to hours and days") + .exclude("SPARK-30793: truncate timestamps before the epoch to seconds and minutes") + .exclude("Gluten - unix_timestamp") + .exclude("Gluten - to_unix_timestamp") + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn].exclude( + "Gluten - SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") + enableSuite[GlutenExpressionsSchemaSuite] + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFileBasedDataSourceSuite] + .exclude("SPARK-23072 Write and read back unicode column names - csv") + .excludeByPrefix("Enabling/disabling ignoreMissingFiles using") + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .exclude("SPARK-25237 compute correct input metrics in FileScanRDD") + .exclude("SPARK-30362: test input metrics for DSV2") + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + .exclude("UDF input_file_name()") + .exclude("Option recursiveFileLookup: disable partition inferring") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + .exclude("SPARK-35669: special char in CSV header with filter pushdown") + .exclude("gluten Spark native readers should respect spark.sql.caseSensitive - parquet") + .exclude("gluten SPARK-25237 compute correct input metrics in FileScanRDD") + .exclude("gluten Option recursiveFileLookup: disable partition inferring") + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenFileSourceCharVarcharTestSuite] + .exclude("char type values should be padded or trimmed: partitioned columns") + .exclude("varchar type values length check and trim: partitioned columns") + .exclude("char/varchar type values length check: partitioned columns of other types") + .exclude("char type comparison: partitioned columns") + enableSuite[GlutenFileSourceSQLInsertTestSuite] + .exclude("SPARK-33474: Support typed literals as partition spec values") + .exclude( + "SPARK-34556: checking duplicate static partition columns should respect case sensitive conf") + enableSuite[GlutenGeneratorFunctionSuite] + .exclude("single explode_outer") + .exclude("single posexplode") + .exclude("single posexplode_outer") + .exclude("explode_outer and other columns") + .exclude("aliased explode_outer") + .exclude("explode_outer on map") + .exclude("explode_outer on map with aliases") + .exclude("inline_outer") + .exclude("SPARK-14986: Outer lateral view with empty generate expression") + .exclude("outer explode()") + .exclude("generator in aggregate expression") + .exclude("SPARK-37947: lateral view _outer()") + enableSuite[GlutenInjectRuntimeFilterSuite].exclude("Merge runtime bloom filters") + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite].exclude( + "SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + enableSuite[GlutenJsonFunctionsSuite] + .exclude("function get_json_object - support single quotes") + .exclude("function get_json_object - null") + .exclude("from_json with option") + .exclude("from_json missing columns") + .exclude("from_json invalid json") + .exclude("from_json array support") + .exclude("to_json with option") + .exclude("roundtrip in to_json and from_json - array") + .exclude("SPARK-19637 Support to_json in SQL") + .exclude("SPARK-19967 Support from_json in SQL") + .exclude("pretty print - roundtrip from_json -> to_json") + .exclude("from_json invalid json - check modes") + .exclude("SPARK-36069: from_json invalid json schema - check field name and field value") + .exclude("corrupt record column in the middle") + .exclude("parse timestamps with locale") + .exclude("from_json - timestamp in micros") + .exclude("SPARK-33134: return partial results only for root JSON objects") + .exclude("SPARK-33907: bad json input with json pruning optimization: GetStructField") + .exclude("SPARK-33907: json pruning optimization with corrupt record field") + enableSuite[GlutenMathFunctionsSuite].exclude("csc").exclude("sec") + enableSuite[GlutenMetadataCacheSuite].exclude( + "SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenSQLQuerySuite] + .exclude("self join with alias in agg") + .exclude("SPARK-3176 Added Parser of SQL LAST()") + .exclude("SPARK-3173 Timestamp support in the parser") + .exclude("SPARK-11111 null-safe join should not use cartesian product") + .exclude("SPARK-3349 partitioning after limit") + .exclude("aggregation with codegen updates peak execution memory") + .exclude("SPARK-10215 Div of Decimal returns null") + .exclude("precision smaller than scale") + .exclude("external sorting updates peak execution memory") + .exclude("run sql directly on files") + .exclude("Struct Star Expansion") + .exclude("Common subexpression elimination") + .exclude( + "SPARK-27619: When spark.sql.legacy.allowHashOnMapType is true, hash can be used on Maptype") + .exclude("SPARK-24940: coalesce and repartition hint") + .exclude("SPARK-25144 'distinct' causes memory leak") + .exclude("SPARK-29239: Subquery should not cause NPE when eliminating subexpression") + .exclude("normalize special floating numbers in subquery") + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + .exclude("SPARK-33593: Vector reader got incorrect data with binary partition value") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + .exclude("SPARK-27442: Spark support read/write parquet file with invalid char in field name") + .exclude("SPARK-37965: Spark support read/write orc file with invalid char in field name") + .exclude("SPARK-38548: try_sum should return null if overflow happens before merging") + .exclude("SPARK-38589: try_avg should return null if overflow happens before merging") + .exclude("SPARK-39548: CreateView will make queries go into inline CTE code path thustrigger a mis-clarified `window definition not found` issue") + .exclude("Gluten - SPARK-33593: Vector reader got incorrect data with binary partition value") + .exclude("Gluten - SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + enableSuite[GlutenStatisticsCollectionSuite] + .exclude("analyze empty table") + .exclude("analyze column command - result verification") + .exclude("column stats collection for null columns") + enableSuite[GlutenStringFunctionsSuite] + .exclude("string regex_replace / regex_extract") + .exclude("string overlay function") + .exclude("binary overlay function") + .exclude("string / binary substring function") + .exclude("string parse_url function") + enableSuite[GlutenSubquerySuite] + .exclude("SPARK-15370: COUNT bug in subquery in subquery in subquery") + .exclude("SPARK-26893: Allow pushdown of partition pruning subquery filters to file source") + .exclude("SPARK-28441: COUNT bug in nested subquery with non-foldable expr") + .exclude("SPARK-28441: COUNT bug with non-foldable expression in Filter condition") + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + .exclude("SPARK-36656: Do not collapse projects with correlate scalar subqueries") + .exclude("Merge non-correlated scalar subqueries from different parent plans") + .exclude("Merge non-correlated scalar subqueries with conflicting names") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite].exclude("cases when literal is max") + enableSuite[GlutenXPathFunctionsSuite] + enableSuite[QueryTestSuite] + enableSuite[GlutenArithmeticExpressionSuite] + .exclude("- (UnaryMinus)") + .exclude("/ (Divide) basic") + .exclude("/ (Divide) for Long and Decimal type") + .exclude("% (Remainder)") + .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") + .exclude("SPARK-36920: Support year-month intervals by ABS") + .exclude("SPARK-36920: Support day-time intervals by ABS") + .exclude("SPARK-36921: Support YearMonthIntervalType by div") + .exclude("SPARK-36921: Support DayTimeIntervalType by div") + enableSuite[GlutenBitwiseExpressionsSuite].exclude("BitGet") + enableSuite[GlutenCastSuite] + .exclude("null cast") + .exclude("cast string to date") + .exclude("cast string to timestamp") + .exclude("cast from boolean") + .exclude("data type casting") + .exclude("cast between string and interval") + .exclude("SPARK-27671: cast from nested null type in struct") + .exclude("Process Infinity, -Infinity, NaN in case insensitive manner") + .exclude("SPARK-22825 Cast array to string") + .exclude("SPARK-33291: Cast array with null elements to string") + .exclude("SPARK-22973 Cast map to string") + .exclude("SPARK-22981 Cast struct to string") + .exclude("SPARK-33291: Cast struct with null elements to string") + .exclude("SPARK-34667: cast year-month interval to string") + .exclude("SPARK-34668: cast day-time interval to string") + .exclude("SPARK-35698: cast timestamp without time zone to string") + .exclude("SPARK-35711: cast timestamp without time zone to timestamp with local time zone") + .exclude("SPARK-35716: cast timestamp without time zone to date type") + .exclude("SPARK-35718: cast date type to timestamp without timezone") + .exclude("SPARK-35719: cast timestamp with local time zone to timestamp without timezone") + .exclude("SPARK-35720: cast string to timestamp without timezone") + .exclude("SPARK-35112: Cast string to day-time interval") + .exclude("SPARK-35111: Cast string to year-month interval") + .exclude("SPARK-35820: Support cast DayTimeIntervalType in different fields") + .exclude("SPARK-35819: Support cast YearMonthIntervalType in different fields") + .exclude("SPARK-35768: Take into account year-month interval fields in cast") + .exclude("SPARK-35735: Take into account day-time interval fields in cast") + .exclude("null cast #2") + .exclude("cast string to date #2") + .exclude("casting to fixed-precision decimals") + .exclude("SPARK-28470: Cast should honor nullOnOverflow property") + .exclude("cast string to boolean II") + .exclude("cast from array II") + .exclude("cast from map II") + .exclude("cast from struct II") + .exclude("cast from date") + .exclude("cast from timestamp II") + .exclude("cast a timestamp before the epoch 1970-01-01 00:00:00Z") + .exclude("SPARK-32828: cast from a derived user-defined type to a base type") + .exclude("SPARK-34727: cast from float II") + .exclude("SPARK-35720: cast invalid string input to timestamp without time zone") + .exclude("SPARK-36924: Cast DayTimeIntervalType to IntegralType") + .exclude("SPARK-36924: Cast IntegralType to DayTimeIntervalType") + .exclude("SPARK-36924: Cast YearMonthIntervalType to IntegralType") + .exclude("SPARK-36924: Cast IntegralType to YearMonthIntervalType") + enableSuite[GlutenCollectionExpressionsSuite] + .exclude("Array and Map Size") + .exclude("MapEntries") + .exclude("Map Concat") + .exclude("MapFromEntries") + .exclude("ArraysOverlap") + .exclude("ArrayJoin") + .exclude("ArraysZip") + .exclude("Sequence of numbers") + .exclude("Sequence of timestamps") + .exclude("Sequence on DST boundaries") + .exclude("Sequence of dates") + .exclude("SPARK-37544: Time zone should not affect date sequence with month interval") + .exclude("SPARK-35088: Accept ANSI intervals by the Sequence expression") + .exclude("SPARK-36090: Support TimestampNTZType in expression Sequence") + .exclude("Sequence with default step") + .exclude("Reverse") + .exclude("elementAt") + .exclude("Flatten") + .exclude("ArrayRepeat") + .exclude("Array remove") + .exclude("Array Distinct") + .exclude("Shuffle") + .exclude("Array Except") + .exclude("Array Except - null handling") + .exclude("SPARK-31980: Start and end equal in month range") + .exclude("SPARK-36639: Start and end equal in month range with a negative step") + .exclude("SPARK-33386: element_at ArrayIndexOutOfBoundsException") + .exclude("SPARK-33460: element_at NoSuchElementException") + .exclude("SPARK-36753: ArrayExcept should handle duplicated Double.NaN and Float.Nan") + .exclude("SPARK-36755: ArraysOverlap hould handle duplicated Double.NaN and Float.Nan") + .exclude( + "SPARK-36740: ArrayMin/ArrayMax/SortArray should handle NaN greater then non-NaN value") + .exclude("SPARK-39184: Avoid ArrayIndexOutOfBoundsException when crossing DST boundary") + enableSuite[GlutenComplexTypeSuite] + .exclude("SPARK-33386: GetArrayItem ArrayIndexOutOfBoundsException") + .exclude("SPARK-33460: GetMapValue NoSuchElementException") + .exclude("GetArrayStructFields") + .exclude("CreateMap") + .exclude("MapFromArrays") + enableSuite[GlutenConditionalExpressionSuite] + .exclude("case when") + .exclude("if/case when - null flags of non-primitive types") + enableSuite[GlutenDateExpressionsSuite] + .exclude("DayOfYear") + .exclude("Year") + .exclude("Quarter") + .exclude("Month") + .exclude("Day / DayOfMonth") + .exclude("Seconds") + .exclude("DayOfWeek") + .exclude("WeekDay") + .exclude("WeekOfYear") + .exclude("DateFormat") + .exclude("Hour") + .exclude("Minute") + .exclude("date add interval") + .exclude("time_add") + .exclude("time_sub") + .exclude("add_months") + .exclude("SPARK-34721: add a year-month interval to a date") + .exclude("months_between") + .exclude("next_day") + .exclude("TruncDate") + .exclude("TruncTimestamp") + .exclude("unsupported fmt fields for trunc/date_trunc results null") + .exclude("from_unixtime") + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + .exclude("to_utc_timestamp") + .exclude("from_utc_timestamp") + .exclude("creating values of DateType via make_date") + .exclude("creating values of Timestamp/TimestampNTZ via make_timestamp") + .exclude("ISO 8601 week-numbering year") + .exclude("extract the seconds part with fraction from timestamps") + .exclude("SPARK-34903: timestamps difference") + .exclude("SPARK-35916: timestamps without time zone difference") + .exclude("SPARK-34896: subtract dates") + .exclude("to_timestamp_ntz") + .exclude("to_timestamp exception mode") + .exclude("SPARK-31896: Handle am-pm timestamp parsing when hour is missing") + .exclude("DATE_FROM_UNIX_DATE") + .exclude("UNIX_DATE") + .exclude("UNIX_SECONDS") + .exclude("UNIX_MILLIS") + .exclude("UNIX_MICROS") + .exclude("TIMESTAMP_SECONDS") + .exclude("TIMESTAMP_MILLIS") + .exclude("TIMESTAMP_MICROS") + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + .exclude("SPARK-34739,SPARK-35889: add a year-month interval to a timestamp") + .exclude("SPARK-34761,SPARK-35889: add a day-time interval to a timestamp") + .exclude("SPARK-37552: convert a timestamp_ntz to another time zone") + .exclude("SPARK-38195: add a quantity of interval units to a timestamp") + .exclude("SPARK-38284: difference between two timestamps in units") + .exclude("Gluten - TIMESTAMP_MICROS") + .exclude("Gluten - unix_timestamp") + .exclude("Gluten - to_unix_timestamp") + enableSuite[GlutenDecimalExpressionSuite].exclude("MakeDecimal") + enableSuite[GlutenHashExpressionsSuite] + .exclude("sha2") + .exclude("murmur3/xxHash64/hive hash: struct") + .exclude("SPARK-30633: xxHash64 with long seed: struct") + .exclude("murmur3/xxHash64/hive hash: struct,arrayOfString:array,arrayOfArrayOfString:array>,arrayOfArrayOfInt:array>,arrayOfStruct:array>,arrayOfUDT:array>") + .exclude("SPARK-30633: xxHash64 with long seed: struct,arrayOfString:array,arrayOfArrayOfString:array>,arrayOfArrayOfInt:array>,arrayOfStruct:array>,arrayOfUDT:array>") + .exclude("murmur3/xxHash64/hive hash: struct,structOfStructOfString:struct>,structOfArray:struct>,structOfUDT:struct>") + .exclude("SPARK-30633: xxHash64 with long seed: struct,structOfStructOfString:struct>,structOfArray:struct>,structOfUDT:struct>") + .exclude("SPARK-30633: xxHash with different type seeds") + .exclude("SPARK-35113: HashExpression support DayTimeIntervalType/YearMonthIntervalType") + .exclude("SPARK-35207: Compute hash consistent between -0.0 and 0.0") + enableSuite[GlutenIntervalExpressionsSuite] + .exclude("years") + .exclude("months") + .exclude("days") + .exclude("hours") + .exclude("minutes") + .exclude("seconds") + .exclude("multiply") + .exclude("divide") + .exclude("make interval") + .exclude("ANSI mode: make interval") + .exclude("SPARK-35130: make day time interval") + .exclude("SPARK-34824: multiply year-month interval by numeric") + .exclude("SPARK-34850: multiply day-time interval by numeric") + .exclude("SPARK-34868: divide year-month interval by numeric") + .exclude("SPARK-34875: divide day-time interval by numeric") + .exclude("ANSI: extract years and months") + .exclude("ANSI: extract days, hours, minutes and seconds") + .exclude("SPARK-35129: make_ym_interval") + .exclude("SPARK-35728: Check multiply/divide of day-time intervals of any fields by numeric") + .exclude("SPARK-35778: Check multiply/divide of year-month intervals of any fields by numeric") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("null") + .exclude("default") + .exclude("decimal") + .exclude("array") + .exclude("seq") + .exclude("map") + .exclude("struct") + .exclude("SPARK-35664: construct literals from java.time.LocalDateTime") + .exclude("SPARK-34605: construct literals from java.time.Duration") + .exclude("SPARK-34605: construct literals from arrays of java.time.Duration") + .exclude("SPARK-34615: construct literals from java.time.Period") + .exclude("SPARK-34615: construct literals from arrays of java.time.Period") + .exclude("SPARK-35871: Literal.create(value, dataType) should support fields") + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenMathExpressionsSuite] + .exclude("csc") + .exclude("sec") + .exclude("cot") + .exclude("tanh") + .exclude("ceil") + .exclude("floor") + .exclude("factorial") + .exclude("rint") + .exclude("expm1") + .exclude("log") + .exclude("log10") + .exclude("bin") + .exclude("log2") + .exclude("unhex") + .exclude("atan2") + .exclude("binary log") + .exclude("round/bround/floor/ceil") + .exclude("Gluten - round/bround/floor/ceil") + .exclude("SPARK-36922: Support ANSI intervals for SIGN/SIGNUM") + .exclude("SPARK-35926: Support YearMonthIntervalType in width-bucket function") + .exclude("SPARK-35925: Support DayTimeIntervalType in width-bucket function") + .exclude("SPARK-37388: width_bucket") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + .exclude("InputFileName") + enableSuite[GlutenNullExpressionsSuite] + .exclude("nanvl") + .exclude("AtLeastNNonNulls") + .exclude("AtLeastNNonNulls should not throw 64KiB exception") + enableSuite[GlutenPredicateSuite] + .exclude("3VL Not") + .exclude("3VL AND") + .exclude("3VL OR") + .exclude("3VL =") + .exclude("basic IN/INSET predicate test") + .exclude("IN with different types") + .exclude("IN/INSET: binary") + .exclude("IN/INSET: struct") + .exclude("IN/INSET: array") + .exclude("BinaryComparison: lessThan") + .exclude("BinaryComparison: LessThanOrEqual") + .exclude("BinaryComparison: GreaterThan") + .exclude("BinaryComparison: GreaterThanOrEqual") + .exclude("BinaryComparison: EqualTo") + .exclude("BinaryComparison: EqualNullSafe") + .exclude("BinaryComparison: null test") + .exclude("EqualTo on complex type") + .exclude("isunknown and isnotunknown") + .exclude("SPARK-32764: compare special double/float values") + .exclude("SPARK-32110: compare special double/float values in array") + .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 '#'") + .exclude("LIKE Pattern ESCAPE '\"'") + .exclude("RLIKE Regular Expression") + .exclude("RegexReplace") + .exclude("RegexExtract") + .exclude("RegexExtractAll") + .exclude("SPLIT") + .exclude(" SPARK -34814: LikeSimplification should handleNULL") + enableSuite[GlutenSortOrderExpressionsSuite].exclude("SortPrefix") + enableSuite[GlutenStringExpressionsSuite] + .exclude("concat") + .exclude("StringComparison") + .exclude("Substring") + .exclude("string substring_index function") + .exclude("SPARK-40213: ascii for Latin-1 Supplement characters") + .exclude("ascii for string") + .exclude("string for ascii") + .exclude("base64/unbase64 for string") + .exclude("encode/decode for string") + .exclude("Levenshtein distance") + .exclude("soundex unit test") + .exclude("replace") + .exclude("overlay for string") + .exclude("overlay for byte array") + .exclude("translate") + .exclude("FORMAT") + .exclude("SPARK-22603: FormatString should not generate codes beyond 64KB") + .exclude("INSTR") + .exclude("LOCATE") + .exclude("LPAD/RPAD") + .exclude("REPEAT") + .exclude("length for string / binary") + .exclude("format_number / FormatNumber") + .exclude("ToNumber: positive tests") + .exclude("ToNumber: negative tests (the input string does not match the format string)") + .exclude("ParseUrl") + .exclude("SPARK-33468: ParseUrl in ANSI mode should fail if input string is not a valid url") + .exclude("Sentences") + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + enableSuite[GlutenDataSourceV2Suite] + .exclude("partitioning reporting") + .exclude("SPARK-33267: push down with condition 'in (..., null)' should not throw NPE") + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenFileDataSourceV2FallBackSuite] + enableSuite[GlutenKeyGroupedPartitioningSuite] + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + enableSuite[GlutenQueryCompilationErrorsSuite] + .exclude("CANNOT_USE_MIXTURE: Using aggregate function with grouped aggregate pandas UDF") + .exclude("UNSUPPORTED_FEATURE: Using pandas UDF aggregate expression with pivot") + enableSuite[GlutenQueryExecutionErrorsSuite] + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + .exclude("UNSUPPORTED_OPERATION - SPARK-38504: can't read TimestampNTZ as TimestampLTZ") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[FallbackStrategiesSuite] + enableSuite[GlutenBroadcastExchangeSuite] + enableSuite[GlutenCoalesceShufflePartitionsSuite] + .exclude( + "determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") + .exclude("determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") + .exclude("determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") + .exclude("determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") + .exclude( + "determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") + .exclude("determining the number of reducers: aggregate operator") + .exclude("determining the number of reducers: join operator") + .exclude("determining the number of reducers: complex query 1") + .exclude("determining the number of reducers: complex query 2") + .exclude("determining the number of reducers: plan already partitioned") + .exclude("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") + .exclude("Do not reduce the number of shuffle partition for repartition") + .exclude("Union two datasets with different pre-shuffle partition number") + .exclude("SPARK-34790: enable IO encryption in AQE partition coalescing") + .exclude( + "Gluten - SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") + .exclude("Gluten - SPARK-34790: enable IO encryption in AQE partition coalescing") + .exclude("Gluten - determining the number of reducers: aggregate operator(minNumPostShufflePartitions: 5)") + .exclude( + "Gluten - determining the number of reducers: join operator(minNumPostShufflePartitions: 5)") + .exclude( + "Gluten - determining the number of reducers: complex query 1(minNumPostShufflePartitions: 5)") + .exclude( + "Gluten - determining the number of reducers: complex query 2(minNumPostShufflePartitions: 5)") + .exclude("Gluten - determining the number of reducers: plan already partitioned(minNumPostShufflePartitions: 5)") + .exclude("Gluten - determining the number of reducers: aggregate operator") + .exclude("Gluten - determining the number of reducers: join operator") + .exclude("Gluten - determining the number of reducers: complex query 1") + .exclude("Gluten - determining the number of reducers: complex query 2") + .exclude("Gluten - determining the number of reducers: plan already partitioned") + enableSuite[GlutenExchangeSuite] + .exclude("shuffling UnsafeRows in exchange") + .exclude("SPARK-23207: Make repartition() generate consistent output") + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .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") + .exclude("Gluten - replace partial hash aggregate with sort aggregate") + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenSQLWindowFunctionSuite] + .exclude("window function: partition and order expressions") + .exclude("window function: expressions in arguments of a window functions") + .exclude( + "window function: multiple window expressions specified by range in a single expression") + .exclude("SPARK-7595: Window will cause resolve failed with self join") + .exclude( + "SPARK-16633: lead/lag should return the default value if the offset row does not exist") + .exclude("lead/lag should respect null values") + .exclude("test with low buffer spill threshold") + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSortSuite] + .exclude("basic sorting using ExternalSort") + .exclude("sort followed by limit") + .exclude("sorting does not crash for large inputs") + .exclude("sorting updates peak execution memory") + .exclude("SPARK-33260: sort order is a Stream") + .exclude("sorting on StringType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on StringType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on StringType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on StringType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on StringType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on StringType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on LongType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on LongType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on LongType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on LongType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on LongType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on LongType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on LongType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on LongType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on IntegerType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on IntegerType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on DecimalType(20,5) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude( + "sorting on DecimalType(20,5) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on DoubleType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on DoubleType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DateType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DateType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DateType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on DateType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DateType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DateType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DateType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on DateType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on BooleanType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on BooleanType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude( + "sorting on DecimalType(38,18) with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude( + "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude( + "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude( + "sorting on DecimalType(38,18) with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on ByteType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on ByteType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on ByteType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on ByteType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on ByteType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on ByteType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on ByteType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on ByteType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on FloatType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on FloatType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on FloatType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on FloatType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on FloatType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on FloatType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on FloatType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on FloatType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on ShortType with nullable=true, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on ShortType with nullable=true, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on ShortType with nullable=true, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on ShortType with nullable=true, sortOrder=List('a DESC NULLS FIRST)") + .exclude("sorting on ShortType with nullable=false, sortOrder=List('a ASC NULLS FIRST)") + .exclude("sorting on ShortType with nullable=false, sortOrder=List('a ASC NULLS LAST)") + .exclude("sorting on ShortType with nullable=false, sortOrder=List('a DESC NULLS LAST)") + .exclude("sorting on ShortType with nullable=false, sortOrder=List('a DESC NULLS FIRST)") + enableSuite[GlutenTakeOrderedAndProjectSuite] + .exclude("TakeOrderedAndProject.doExecute without project") + .exclude("TakeOrderedAndProject.doExecute with project") + enableSuite[GlutenAdaptiveQueryExecSuite] + .exclude("Change merge join to broadcast join") + .exclude("Reuse the parallelism of coalesced shuffle in local shuffle read") + .exclude("Reuse the default parallelism in local shuffle read") + .exclude("Empty stage coalesced to 1-partition RDD") + .exclude("Scalar subquery") + .exclude("Scalar subquery in later stages") + .exclude("multiple joins") + .exclude("multiple joins with aggregate") + .exclude("multiple joins with aggregate 2") + .exclude("Exchange reuse") + .exclude("Exchange reuse with subqueries") + .exclude("Exchange reuse across subqueries") + .exclude("Subquery reuse") + .exclude("Broadcast exchange reuse across subqueries") + .exclude("Change merge join to broadcast join without local shuffle read") + .exclude( + "Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude("SPARK-37753: Allow changing outer join to broadcast join even if too many empty partitions on broadcast side") + .exclude("SPARK-29544: adaptive skew join with different join types") + .exclude("SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") + .exclude("metrics of the shuffle read") + .exclude("SPARK-31220, SPARK-32056: repartition by expression with AQE") + .exclude("SPARK-31220, SPARK-32056: repartition by range with AQE") + .exclude("SPARK-31220, SPARK-32056: repartition using sql and hint with AQE") + .exclude("SPARK-32753: Only copy tags to node with no tags") + .exclude("Logging plan changes for AQE") + .exclude("SPARK-33551: Do not use AQE shuffle read for repartition") + .exclude("SPARK-34091: Batch shuffle fetch in AQE partition coalescing") + .exclude("SPARK-34899: Use origin plan if we can not coalesce shuffle partition") + .exclude("SPARK-34980: Support coalesce partition through union") + .exclude("SPARK-35239: Coalesce shuffle partition should handle empty input RDD") + .exclude("SPARK-35264: Support AQE side broadcastJoin threshold") + .exclude("SPARK-35264: Support AQE side shuffled hash join formula") + .exclude("SPARK-35650: Coalesce number of partitions by AEQ") + .exclude("SPARK-35650: Use local shuffle read if can not coalesce number of partitions") + .exclude("SPARK-35725: Support optimize skewed partitions in RebalancePartitions") + .exclude("SPARK-35888: join with a 0-partition table") + .exclude("SPARK-33832: Support optimize skew join even if introduce extra shuffle") + .exclude("SPARK-35968: AQE coalescing should not produce too small partitions by default") + .exclude("SPARK-35794: Allow custom plugin for cost evaluator") + .exclude("SPARK-36020: Check logical link in remove redundant projects") + .exclude("SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") + .exclude("SPARK-37063: OptimizeSkewInRebalancePartitions support optimize non-root node") + .exclude("SPARK-37357: Add small partition factor for rebalance partitions") + .exclude("SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") + .exclude("SPARK-37328: skew join with 3 tables") + .exclude("SPARK-39915: Dataset.repartition(N) may not create N partitions") + .exclude("gluten Change broadcast join to merge join") + .exclude("gluten Empty stage coalesced to 1-partition RDD") + .exclude("gluten Avoid changing merge join to broadcast join if too many empty partitions on build plan") + .exclude("gluten SPARK-30524: Do not optimize skew join if introduce additional shuffle") + .exclude("gluten SPARK-33551: Do not use AQE shuffle read for repartition") + .exclude("gluten SPARK-35264: Support AQE side broadcastJoin threshold") + .exclude("gluten SPARK-35264: Support AQE side shuffled hash join formula") + .exclude("gluten SPARK-35725: Support optimize skewed partitions in RebalancePartitions") + .exclude( + "gluten SPARK-35968: AQE coalescing should not produce too small partitions by default") + .exclude( + "gluten SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenFileFormatWriterSuite].excludeByPrefix( + "empty file should be skipped while write to file") + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + .exclude("metadata struct (json): file metadata in streaming") + .exclude("metadata struct (parquet): file metadata in streaming") + enableSuite[GlutenFileSourceStrategySuite] + .exclude("unpartitioned table, single partition") + .exclude("partitioned table - after scan filters") + .exclude("SPARK-32019: Add spark.sql.files.minPartitionNum config") + .exclude( + "SPARK-32352: Partially push down support data filter if it mixed in partition filters") + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenHeaderCSVReadSchemaSuite] + .exclude("append column at the end") + .exclude("hide column at the end") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + .exclude("change column type from float to decimal") + .exclude("change column type from double to decimal") + .exclude("read float, double, decimal together") + .exclude("read as string") + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenMergedOrcReadSchemaSuite] + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenOrcReadSchemaSuite] + enableSuite[GlutenOrcV1AggregatePushDownSuite].exclude( + "aggregate push down - different data types") + enableSuite[GlutenOrcV2AggregatePushDownSuite].exclude( + "aggregate push down - different data types") + enableSuite[GlutenParquetCodecSuite] + enableSuite[GlutenParquetReadSchemaSuite] + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV2AggregatePushDownSuite] + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenBinaryFileFormatSuite] + .exclude("column pruning - non-readable file") + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenJsonLegacyTimeParserSuite] + .exclude("Complex field and type inferring") + .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") + .exclude("SPARK-4228 DataFrame to JSON") + .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") + .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + enableSuite[GlutenJsonSuite] + .exclude("Complex field and type inferring") + .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") + .exclude("SPARK-4228 DataFrame to JSON") + .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") + .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + enableSuite[GlutenJsonV1Suite] + .exclude("Complex field and type inferring") + .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") + .exclude("SPARK-4228 DataFrame to JSON") + .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") + .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + enableSuite[GlutenJsonV2Suite] + .exclude("Complex field and type inferring") + .exclude("Loading a JSON dataset primitivesAsString returns complex fields as strings") + .exclude("SPARK-4228 DataFrame to JSON") + .exclude("SPARK-18352: Handle multi-line corrupt documents (PERMISSIVE)") + .exclude("SPARK-37360: Write and infer TIMESTAMP_NTZ values with a non-default pattern") + .exclude("SPARK-37360: Timestamp type inference for a column with TIMESTAMP_NTZ values") + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite].exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + enableSuite[GlutenOrcQuerySuite] + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + enableSuite[GlutenOrcSourceSuite] + .exclude("SPARK-24322 Fix incorrect workaround for bug in java.sql.Timestamp") + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") + .exclude("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .exclude("SPARK-37812: Reuse result row when deserializing a struct") + .exclude("Gluten - SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("Gluten - SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("Gluten - SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("Gluten - SPARK-31284, SPARK-31423: rebasing timestamps in write") + .exclude("Gluten - SPARK-34862: Support ORC vectorized reader for nested column") + enableSuite[GlutenOrcV1FilterSuite].exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + enableSuite[GlutenOrcV1QuerySuite] + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + enableSuite[GlutenOrcV1SchemaPruningSuite] + .exclude( + "Spark vectorized reader - without partition data column - select only top-level fields") + .exclude("Spark vectorized reader - with partition data column - select only top-level fields") + .exclude("Non-vectorized reader - without partition data column - select only top-level fields") + .exclude("Non-vectorized reader - with partition data column - select only top-level fields") + .exclude("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude( + "Spark vectorized reader - without partition data column - select only input_file_name()") + .exclude("Spark vectorized reader - with partition data column - select only input_file_name()") + .exclude( + "Non-vectorized reader - without partition data column - select only input_file_name()") + .exclude("Non-vectorized reader - with partition data column - select only input_file_name()") + .exclude("Spark vectorized reader - without partition data column - select only expressions without references") + .exclude("Spark vectorized reader - with partition data column - select only expressions without references") + .exclude("Non-vectorized reader - without partition data column - select only expressions without references") + .exclude("Non-vectorized reader - with partition data column - select only expressions without references") + .exclude( + "Spark vectorized reader - without partition data column - select a single complex field") + .exclude("Spark vectorized reader - with partition data column - select a single complex field") + .exclude( + "Non-vectorized reader - without partition data column - select a single complex field") + .exclude("Non-vectorized reader - with partition data column - select a single complex field") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - without partition data column - empty schema intersection") + .exclude("Spark vectorized reader - with partition data column - empty schema intersection") + .exclude("Non-vectorized reader - without partition data column - empty schema intersection") + .exclude("Non-vectorized reader - with partition data column - empty schema intersection") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - without partition data column - select nullable complex field and having is not null predicate") + .exclude("Spark vectorized reader - with partition data column - select nullable complex field and having is not null predicate") + .exclude("Non-vectorized reader - without partition data column - select nullable complex field and having is not null predicate") + .exclude("Non-vectorized reader - with partition data column - select nullable complex field and having is not null predicate") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and in clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and in clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and in clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and in clause") + .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function") + .exclude( + "Non-vectorized reader - with partition data column - select nested field in window function") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .exclude("Spark vectorized reader - with partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - without partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - with partition data column - select nested field in Sort") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - with partition data column - select nested field in Expand") + .exclude( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .exclude("Non-vectorized reader - with partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + enableSuite[GlutenOrcV2QuerySuite] + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + enableSuite[GlutenOrcV2SchemaPruningSuite] + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + enableSuite[GlutenParquetColumnIndexSuite] + .exclude("test reading unaligned pages - test all types") + .exclude("test reading unaligned pages - test all types (dict encode)") + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite].exclude("All Types Dictionary").exclude("All Types Null") + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + enableSuite[GlutenParquetFileFormatV2Suite] + enableSuite[GlutenParquetIOSuite] + .exclude("Standard mode - nested map with struct as key type") + .exclude("Legacy mode - nested map with struct as key type") + .exclude("vectorized reader: missing all struct fields") + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + .exclude("SPARK-35640: int as long should throw schema incompatible error") + .exclude("SPARK-36726: test incorrect Parquet row group file offset") + enableSuite[GlutenParquetInteroperabilitySuite].exclude("parquet timestamp conversion") + enableSuite[GlutenParquetProtobufCompatibilitySuite].exclude("struct with unannotated array") + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + .exclude( + "SPARK-31159, SPARK-37705: compatibility with Spark 2.4/3.2 in reading dates/timestamps") + .exclude("SPARK-31159, SPARK-37705: rebasing timestamps in write") + .exclude("SPARK-31159: rebasing dates in write") + .exclude("SPARK-35427: datetime rebasing in the EXCEPTION mode") + .exclude("gluten SPARK-31159: rebasing dates in write") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + .exclude( + "SPARK-31159, SPARK-37705: compatibility with Spark 2.4/3.2 in reading dates/timestamps") + .exclude("SPARK-31159, SPARK-37705: rebasing timestamps in write") + .exclude("SPARK-31159: rebasing dates in write") + .exclude("SPARK-35427: datetime rebasing in the EXCEPTION mode") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + enableSuite[GlutenParquetThriftCompatibilitySuite] + .exclude("Read Parquet file generated by parquet-thrift") + .exclude("SPARK-10136 list of primitive list") + enableSuite[GlutenParquetV1FilterSuite] + .exclude("filter pushdown - date") + .exclude("filter pushdown - timestamp") + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("filter pushdown - StringStartsWith") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-38825: in and notIn filters") + .exclude("Gluten - SPARK-25207: exception when duplicate fields in case-insensitive mode") + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + .exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping") + .exclude("Various partition value types") + .exclude("Various inferred partition value types") + .exclude( + "SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") + .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV1QuerySuite] + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + .exclude("SPARK-34212 Parquet should read decimals correctly") + enableSuite[GlutenParquetV1SchemaPruningSuite] + .exclude( + "Spark vectorized reader - without partition data column - select only top-level fields") + .exclude("Spark vectorized reader - with partition data column - select only top-level fields") + .exclude("Non-vectorized reader - without partition data column - select only top-level fields") + .exclude("Non-vectorized reader - with partition data column - select only top-level fields") + .exclude("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude( + "Spark vectorized reader - without partition data column - select only input_file_name()") + .exclude("Spark vectorized reader - with partition data column - select only input_file_name()") + .exclude( + "Non-vectorized reader - without partition data column - select only input_file_name()") + .exclude("Non-vectorized reader - with partition data column - select only input_file_name()") + .exclude("Spark vectorized reader - without partition data column - select only expressions without references") + .exclude("Spark vectorized reader - with partition data column - select only expressions without references") + .exclude("Non-vectorized reader - without partition data column - select only expressions without references") + .exclude("Non-vectorized reader - with partition data column - select only expressions without references") + .exclude( + "Spark vectorized reader - without partition data column - select a single complex field") + .exclude("Spark vectorized reader - with partition data column - select a single complex field") + .exclude( + "Non-vectorized reader - without partition data column - select a single complex field") + .exclude("Non-vectorized reader - with partition data column - select a single complex field") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - without partition data column - empty schema intersection") + .exclude("Spark vectorized reader - with partition data column - empty schema intersection") + .exclude("Non-vectorized reader - without partition data column - empty schema intersection") + .exclude("Non-vectorized reader - with partition data column - empty schema intersection") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - without partition data column - select nullable complex field and having is not null predicate") + .exclude("Spark vectorized reader - with partition data column - select nullable complex field and having is not null predicate") + .exclude("Non-vectorized reader - without partition data column - select nullable complex field and having is not null predicate") + .exclude("Non-vectorized reader - with partition data column - select nullable complex field and having is not null predicate") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and in clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and in clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and in clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and in clause") + .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field and having is null predicate on another deep nested complex field") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Spark vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Non-vectorized reader - without partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Non-vectorized reader - with partition data column - SPARK-34638: nested column prune on generator output") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function") + .exclude( + "Non-vectorized reader - with partition data column - select nested field in window function") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .exclude("Spark vectorized reader - with partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - without partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - with partition data column - select nested field in Sort") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - with partition data column - select nested field in Expand") + .exclude( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .exclude("Non-vectorized reader - with partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + enableSuite[GlutenParquetV2FilterSuite] + .exclude("filter pushdown - date") + .exclude("filter pushdown - timestamp") + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("filter pushdown - StringStartsWith") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + .exclude("Support Parquet column index") + .exclude("SPARK-34562: Bloom filter push down") + .exclude("SPARK-38825: in and notIn filters") + .exclude("Gluten - SPARK-25207: exception when duplicate fields in case-insensitive mode") + .exclude("Gluten - filter pushdown - date") + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + .exclude("SPARK-7847: Dynamic partition directory path escaping and unescaping") + .exclude("Various partition value types") + .exclude("Various inferred partition value types") + .exclude( + "SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") + .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + enableSuite[GlutenParquetV2QuerySuite] + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + .exclude("SPARK-34212 Parquet should read decimals correctly") + enableSuite[GlutenParquetV2SchemaPruningSuite] + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and in where clause") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and in where clause") + .exclude("Spark vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - without partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Non-vectorized reader - with partition data column - select one complex field and having is null predicate on another complex field") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Spark vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after repartition by expression") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after repartition by expression") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude( + "Case-insensitive parser - mixed-case schema - filter with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenEnsureRequirementsSuite] + .exclude("reorder should handle PartitioningCollection") + .exclude("SPARK-35675: EnsureRequirements remove shuffle should respect PartitioningCollection") + enableSuite[GlutenBroadcastJoinSuite] + .exclude("unsafe broadcast hash join updates peak execution memory") + .exclude("unsafe broadcast hash outer join updates peak execution memory") + .exclude("unsafe broadcast left semi join updates peak execution memory") + .exclude("SPARK-23192: broadcast hint should be retained after using the cached data") + .exclude("SPARK-23214: cached data should not carry extra hint info") + .exclude("broadcast hint in SQL") + .exclude("Broadcast timeout") + .exclude("broadcast join where streamed side's output partitioning is HashPartitioning") + .exclude("broadcast join where streamed side's output partitioning is PartitioningCollection") + .exclude("BroadcastHashJoinExec output partitioning size should be limited with a config") + .exclude("SPARK-37742: join planning shouldn't read invalid InMemoryRelation stats") + enableSuite[GlutenExistenceJoinSuite] + .exclude("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test single condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test single condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left semi join using BroadcastHashJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build left") + .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left semi join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .exclude("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test composed condition (equal & non-equal) for left semi join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test composed condition (equal & non-equal) for left semi join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test single condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test single condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left anti join using BroadcastHashJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build left") + .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .exclude("test single unique condition (equal) for left anti join using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + .exclude("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test composed condition (equal & non-equal) test for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test composed condition (equal & non-equal) test for left anti join using SortMergeJoin (whole-stage-codegen on)") + .exclude("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("test composed unique condition (both non-equal) for left anti join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen off)") + .exclude("test composed unique condition (both non-equal) for left anti join using SortMergeJoin (whole-stage-codegen on)") + enableSuite[GlutenInnerJoinSuite] + .exclude( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .exclude( + "inner join, one match per row using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .exclude( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .exclude( + "inner join, one match per row using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .exclude("inner join, one match per row using SortMergeJoin (whole-stage-codegen off)") + .exclude("inner join, one match per row using SortMergeJoin (whole-stage-codegen on)") + .exclude( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .exclude( + "inner join, multiple matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .exclude( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .exclude( + "inner join, multiple matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .exclude("inner join, multiple matches using SortMergeJoin (whole-stage-codegen off)") + .exclude("inner join, multiple matches using SortMergeJoin (whole-stage-codegen on)") + .exclude("inner join, no matches using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .exclude("inner join, no matches using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .exclude( + "inner join, no matches using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .exclude("inner join, no matches using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .exclude("inner join, no matches using SortMergeJoin (whole-stage-codegen off)") + .exclude("inner join, no matches using SortMergeJoin (whole-stage-codegen on)") + .exclude("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .exclude("inner join, null safe using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .exclude("inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .exclude("inner join, null safe using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .exclude("inner join, null safe using SortMergeJoin (whole-stage-codegen off)") + .exclude("inner join, null safe using SortMergeJoin (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=left) (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using BroadcastHashJoin (build=right) (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=left) (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using ShuffledHashJoin (build=right) (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using SortMergeJoin (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using CartesianProduct") + .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build left (whole-stage-codegen on)") + .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen off)") + .exclude("SPARK-15822 - test structs as keys using BroadcastNestedLoopJoin build right (whole-stage-codegen on)") + enableSuite[GlutenOuterJoinSuite] + .exclude("basic left outer join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("basic left outer join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("basic left outer join using SortMergeJoin (whole-stage-codegen off)") + .exclude("basic left outer join using SortMergeJoin (whole-stage-codegen on)") + .exclude("basic right outer join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("basic right outer join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("basic right outer join using SortMergeJoin (whole-stage-codegen off)") + .exclude("basic right outer join using SortMergeJoin (whole-stage-codegen on)") + .exclude("basic full outer join using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("basic full outer join using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("basic full outer join using SortMergeJoin (whole-stage-codegen off)") + .exclude("basic full outer join using SortMergeJoin (whole-stage-codegen on)") + .exclude("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("left outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("left outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .exclude("left outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .exclude("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("right outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("right outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .exclude("right outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + .exclude("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen off)") + .exclude("full outer join with unique keys using ShuffledHashJoin (whole-stage-codegen on)") + .exclude("full outer join with unique keys using SortMergeJoin (whole-stage-codegen off)") + .exclude("full outer join with unique keys using SortMergeJoin (whole-stage-codegen on)") + enableSuite[GlutenCustomerExtensionSuite] + enableSuite[GlutenSessionExtensionSuite] + enableSuite[GlutenFallbackSuite] + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude( + "SPARK-19122 Re-order join predicates if they match with the child's output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + .exclude( + "bucket coalescing is applied when join expressions match with partitioning expressions") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + enableSuite[GlutenCreateTableAsSelectSuite] + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenInsertSuite] + enableSuite[GlutenPartitionedWriteSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenTableScanSuite] + .exclude("Schema and all fields") + .exclude("SELECT count(*) FROM tableWithSchema") + .exclude("SELECT `string$%Field` FROM tableWithSchema") + .exclude("SELECT int_Field FROM tableWithSchema WHERE int_Field < 5") + .exclude("SELECT `longField_:,<>=+/~^` * 2 FROM tableWithSchema") + .exclude( + "SELECT structFieldSimple.key, arrayFieldSimple[1] FROM tableWithSchema a where int_Field=1") + .exclude("SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema") + enableSuite[SparkFunctionStatistics] + +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala new file mode 100644 index 000000000000..40b068dc0aed --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/io/glutenproject/utils/velox/VeloxTestSettings.scala @@ -0,0 +1,1206 @@ +/* + * 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 io.glutenproject.utils.velox + +import io.glutenproject.utils.BackendTestSettings + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.{GlutenArithmeticExpressionSuite, GlutenBitwiseExpressionsSuite, GlutenCastSuite, GlutenCollectionExpressionsSuite, GlutenComplexTypeSuite, GlutenConditionalExpressionSuite, GlutenDateExpressionsSuite, GlutenDecimalExpressionSuite, GlutenHashExpressionsSuite, GlutenIntervalExpressionsSuite, GlutenLiteralExpressionSuite, GlutenMathExpressionsSuite, GlutenMiscExpressionsSuite, GlutenNondeterministicSuite, GlutenNullExpressionsSuite, GlutenPredicateSuite, GlutenRandomSuite, GlutenRegexpExpressionsSuite, GlutenSortOrderExpressionsSuite, GlutenStringExpressionsSuite} +import org.apache.spark.sql.connector.{GlutenDataSourceV2DataFrameSessionCatalogSuite, GlutenDataSourceV2DataFrameSuite, GlutenDataSourceV2FunctionSuite, GlutenDataSourceV2SQLSessionCatalogSuite, GlutenDataSourceV2SQLSuiteV1Filter, GlutenDataSourceV2SQLSuiteV2Filter, GlutenDataSourceV2Suite, GlutenDeleteFromTableSuite, GlutenFileDataSourceV2FallBackSuite, 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, GlutenReplaceHashWithSortAggSuite, GlutenReuseExchangeAndSubquerySuite, GlutenSameResultSuite, GlutenSortSuite, GlutenSQLWindowFunctionSuite, GlutenTakeOrderedAndProjectSuite} +import org.apache.spark.sql.execution.adaptive.GlutenAdaptiveQueryExecSuite +import org.apache.spark.sql.execution.datasources.{GlutenBucketingUtilsSuite, GlutenCSVReadSchemaSuite, GlutenDataSourceStrategySuite, GlutenDataSourceSuite, GlutenFileFormatWriterSuite, GlutenFileIndexSuite, GlutenFileMetadataStructSuite, GlutenFileSourceStrategySuite, GlutenHadoopFileLinesReaderSuite, GlutenHeaderCSVReadSchemaSuite, GlutenJsonReadSchemaSuite, GlutenMergedOrcReadSchemaSuite, GlutenMergedParquetReadSchemaSuite, GlutenOrcCodecSuite, GlutenOrcReadSchemaSuite, GlutenOrcV1AggregatePushDownSuite, GlutenOrcV2AggregatePushDownSuite, GlutenParquetCodecSuite, GlutenParquetReadSchemaSuite, GlutenParquetV1AggregatePushDownSuite, GlutenParquetV2AggregatePushDownSuite, GlutenPathFilterStrategySuite, GlutenPathFilterSuite, GlutenPruneFileSourcePartitionsSuite, GlutenVectorizedOrcReadSchemaSuite, GlutenVectorizedParquetReadSchemaSuite} +import org.apache.spark.sql.execution.datasources.binaryfile.GlutenBinaryFileFormatSuite +import org.apache.spark.sql.execution.datasources.csv.{GlutenCSVLegacyTimeParserSuite, GlutenCSVv1Suite, GlutenCSVv2Suite} +import org.apache.spark.sql.execution.datasources.exchange.GlutenValidateRequirementsSuite +import org.apache.spark.sql.execution.datasources.json.{GlutenJsonLegacyTimeParserSuite, GlutenJsonV1Suite, GlutenJsonV2Suite} +import org.apache.spark.sql.execution.datasources.orc.{GlutenOrcColumnarBatchReaderSuite, GlutenOrcFilterSuite, GlutenOrcPartitionDiscoverySuite, GlutenOrcSourceSuite, GlutenOrcV1FilterSuite, GlutenOrcV1PartitionDiscoverySuite, GlutenOrcV1QuerySuite, GlutenOrcV1SchemaPruningSuite, GlutenOrcV2QuerySuite, GlutenOrcV2SchemaPruningSuite} +import org.apache.spark.sql.execution.datasources.parquet.{GlutenParquetColumnIndexSuite, GlutenParquetCompressionCodecPrecedenceSuite, GlutenParquetDeltaByteArrayEncodingSuite, GlutenParquetDeltaEncodingInteger, GlutenParquetDeltaEncodingLong, GlutenParquetDeltaLengthByteArrayEncodingSuite, GlutenParquetEncodingSuite, GlutenParquetFieldIdIOSuite, GlutenParquetFileFormatV1Suite, GlutenParquetFileFormatV2Suite, GlutenParquetInteroperabilitySuite, GlutenParquetIOSuite, GlutenParquetProtobufCompatibilitySuite, GlutenParquetRebaseDatetimeV1Suite, GlutenParquetRebaseDatetimeV2Suite, GlutenParquetSchemaInferenceSuite, GlutenParquetSchemaSuite, GlutenParquetThriftCompatibilitySuite, GlutenParquetV1FilterSuite, GlutenParquetV1PartitionDiscoverySuite, GlutenParquetV1QuerySuite, GlutenParquetV1SchemaPruningSuite, GlutenParquetV2FilterSuite, GlutenParquetV2PartitionDiscoverySuite, GlutenParquetV2QuerySuite, GlutenParquetV2SchemaPruningSuite, GlutenParquetVectorizedSuite} +import org.apache.spark.sql.execution.datasources.text.{GlutenTextV1Suite, GlutenTextV2Suite} +import org.apache.spark.sql.execution.datasources.v2.{GlutenDataSourceV2StrategySuite, GlutenFileTableSuite, GlutenV2PredicateSuite} +import org.apache.spark.sql.execution.exchange.GlutenEnsureRequirementsSuite +import org.apache.spark.sql.execution.joins.{GlutenExistenceJoinSuite, GlutenInnerJoinSuite, GlutenOuterJoinSuite} +import org.apache.spark.sql.extension.{GlutenSessionExtensionSuite, TestFileSourceScanExecTransformer} +import org.apache.spark.sql.gluten.GlutenFallbackSuite +import org.apache.spark.sql.hive.execution.GlutenHiveSQLQuerySuite +import org.apache.spark.sql.sources.{GlutenBucketedReadWithoutHiveSupportSuite, GlutenBucketedWriteWithoutHiveSupportSuite, GlutenCreateTableAsSelectSuite, GlutenDDLSourceLoadSuite, GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite, GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE, GlutenExternalCommandRunnerSuite, GlutenFilteredScanSuite, GlutenFiltersSuite, GlutenInsertSuite, GlutenPartitionedWriteSuite, GlutenPathOptionSuite, GlutenPrunedScanSuite, GlutenResolvedDataSourceSuite, GlutenSaveLoadSuite, GlutenTableScanSuite} + +// Some settings' line length exceeds 100 +// scalastyle:off line.size.limit + +class VeloxTestSettings extends BackendTestSettings { + enableSuite[GlutenStringFunctionsSuite] + // TODO: support limit and regex pattern + .exclude("string split function with no limit") + .exclude("string split function with limit explicitly set to 0") + .exclude("string split function with positive limit") + .exclude("string split function with negative limit") + enableSuite[GlutenBloomFilterAggregateQuerySuite] + // fallback might_contain, the input argument binary is not same with vanilla spark + .exclude("Test NULL inputs for might_contain") + enableSuite[GlutenDataSourceV2DataFrameSessionCatalogSuite] + enableSuite[GlutenDataSourceV2DataFrameSuite] + enableSuite[GlutenDataSourceV2FunctionSuite] + enableSuite[GlutenDataSourceV2SQLSessionCatalogSuite] + enableSuite[GlutenDataSourceV2SQLSuiteV1Filter] + enableSuite[GlutenDataSourceV2SQLSuiteV2Filter] + enableSuite[GlutenDataSourceV2Suite] + // Rewrite the following test in GlutenDataSourceV2Suite. + .exclude("partitioning reporting") + .exclude("ordering and partitioning reporting") + enableSuite[GlutenDeleteFromTableSuite] + enableSuite[GlutenFileDataSourceV2FallBackSuite] + enableSuite[GlutenKeyGroupedPartitioningSuite] + // NEW SUITE: disable as they check vanilla spark plan + .exclude("partitioned join: number of buckets mismatch should trigger shuffle") + .exclude("partitioned join: only one side reports partitioning") + .exclude("partitioned join: join with two partition keys and different # of partition keys") + .excludeByPrefix("SPARK-41413: partitioned join:") + .exclude("SPARK-42038: partially clustered: with different partition keys and both sides partially clustered") + .exclude("SPARK-42038: partially clustered: with different partition keys and missing keys on left-hand side") + .exclude("SPARK-42038: partially clustered: with different partition keys and missing keys on right-hand side") + .exclude("SPARK-42038: partially clustered: left outer join") + .exclude("SPARK-42038: partially clustered: right outer join") + .exclude("SPARK-42038: partially clustered: full outer join is not applicable") + .exclude("SPARK-42038: partially clustered: with dynamic partition filtering") + enableSuite[GlutenLocalScanSuite] + enableSuite[GlutenMetadataColumnSuite] + enableSuite[GlutenSupportsCatalogOptionsSuite] + enableSuite[GlutenTableCapabilityCheckSuite] + enableSuite[GlutenWriteDistributionAndOrderingSuite] + .exclude("ordered distribution and sort with same exprs: append") + .exclude("ordered distribution and sort with same exprs: overwrite") + .exclude("ordered distribution and sort with same exprs: overwriteDynamic") + .exclude("clustered distribution and sort with same exprs: append") + .exclude("clustered distribution and sort with same exprs: overwrite") + .exclude("clustered distribution and sort with same exprs: overwriteDynamic") + .exclude("clustered distribution and sort with extended exprs: append") + .exclude("clustered distribution and sort with extended exprs: overwrite") + .exclude("clustered distribution and sort with extended exprs: overwriteDynamic") + .exclude("ordered distribution and sort with manual global sort: append") + .exclude("ordered distribution and sort with manual global sort: overwrite") + .exclude("ordered distribution and sort with manual global sort: overwriteDynamic") + .exclude("ordered distribution and sort with incompatible global sort: append") + .exclude("ordered distribution and sort with incompatible global sort: overwrite") + .exclude("ordered distribution and sort with incompatible global sort: overwriteDynamic") + .exclude("ordered distribution and sort with manual local sort: append") + .exclude("ordered distribution and sort with manual local sort: overwrite") + .exclude("ordered distribution and sort with manual local sort: overwriteDynamic") + .exclude("clustered distribution and local sort with manual global sort: append") + .exclude("clustered distribution and local sort with manual global sort: overwrite") + .exclude("clustered distribution and local sort with manual global sort: overwriteDynamic") + .exclude("clustered distribution and local sort with manual local sort: append") + .exclude("clustered distribution and local sort with manual local sort: overwrite") + .exclude("clustered distribution and local sort with manual local sort: overwriteDynamic") + + enableSuite[GlutenQueryCompilationErrorsDSv2Suite] + enableSuite[GlutenQueryCompilationErrorsSuite] + enableSuite[GlutenQueryExecutionErrorsSuite] + // NEW SUITE: disable as it expects exception which doesn't happen when offloaded to gluten + .exclude( + "INCONSISTENT_BEHAVIOR_CROSS_VERSION: compatibility with Spark 2.4/3.2 in reading/writing dates") + .exclude("FAILED_EXECUTE_UDF: execute user defined function") + .exclude("UNRECOGNIZED_SQL_TYPE: unrecognized SQL type -100") + .exclude("INVALID_BUCKET_FILE: error if there exists any malformed bucket files") + .excludeByPrefix("SCALAR_SUBQUERY_TOO_MANY_ROWS:") + .excludeByPrefix("UNSUPPORTED_FEATURE.MULTI_ACTION_ALTER:") + enableSuite[GlutenQueryParsingErrorsSuite] + enableSuite[GlutenArithmeticExpressionSuite] + .exclude( + "% (Remainder)" // Velox will throw exception when right is zero, need fallback + ) + enableSuite[GlutenBitwiseExpressionsSuite] + enableSuite[GlutenCastSuite] + .exclude( + "Process Infinity, -Infinity, NaN in case insensitive manner" // +inf not supported in folly. + ) + // Timezone. + .exclude("SPARK-35711: cast timestamp without time zone to timestamp with local time zone") + // Timezone. + .exclude("SPARK-35719: cast timestamp with local time zone to timestamp without timezone") + // User defined type. + .exclude("SPARK-32828: cast from a derived user-defined type to a base type") + enableSuite[GlutenCollectionExpressionsSuite] + .exclude("Map Concat") + .exclude("Shuffle") + // TODO: ArrayDistinct should handle duplicated Double.NaN + .excludeByPrefix("SPARK-36741") + // TODO: ArrayIntersect should handle duplicated Double.NaN + .excludeByPrefix("SPARK-36754") + .exclude("Concat") + enableSuite[GlutenConditionalExpressionSuite] + enableSuite[GlutenDateExpressionsSuite] + // Has exception in fallback execution when we use resultDF.collect in evaluation. + .exclude("TIMESTAMP_MICROS") + // Replaced by a gluten test to pass timezone through config. + .exclude("unix_timestamp") + // Replaced by a gluten test to pass timezone through config. + .exclude("to_unix_timestamp") + // Unsupported format: yyyy-MM-dd HH:mm:ss.SSS + .exclude("SPARK-33498: GetTimestamp,UnixTimestamp,ToUnixTimestamp with parseError") + // Replaced by a gluten test to pass timezone through config. + .exclude("DateFormat") + enableSuite[GlutenDecimalExpressionSuite] + enableSuite[GlutenHashExpressionsSuite] + enableSuite[GlutenIntervalExpressionsSuite] + enableSuite[GlutenJsonFunctionsSuite] + // Velox does not support single quotes in get_json_object function. + .exclude("function get_json_object - support single quotes") + enableSuite[GlutenLiteralExpressionSuite] + .exclude("default") + // FIXME(yma11): ObjectType is not covered in RowEncoder/Serializer in vanilla spark + .exclude("SPARK-37967: Literal.create support ObjectType") + enableSuite[GlutenMathExpressionsSuite] + // Spark round UT for round(3.1415,3) is not correct. + .exclude("round/bround/floor/ceil") + enableSuite[GlutenMiscExpressionsSuite] + enableSuite[GlutenNondeterministicSuite] + .exclude("MonotonicallyIncreasingID") + .exclude("SparkPartitionID") + enableSuite[GlutenNullExpressionsSuite] + enableSuite[GlutenPredicateSuite] + enableSuite[GlutenRandomSuite] + .exclude("random") + .exclude("SPARK-9127 codegen with long seed") + enableSuite[GlutenRegexpExpressionsSuite] + enableSuite[GlutenSortOrderExpressionsSuite] + enableSuite[GlutenStringExpressionsSuite] + .exclude("concat") + enableSuite[GlutenAdaptiveQueryExecSuite] + .includeByPrefix( + "gluten", + "SPARK-29906", +// "SPARK-30291", + "SPARK-30403", + "SPARK-30719", + "SPARK-31384", + "SPARK-30953", + "SPARK-31658", + "SPARK-32717", + "SPARK-32649", + "SPARK-34533", + "SPARK-34781", + "SPARK-35585", + "SPARK-32932", + "SPARK-33494", +// "SPARK-33933", + "SPARK-31220", + "SPARK-35874", + "SPARK-39551" + ) + .include( + "Union/Except/Intersect queries", + "Subquery de-correlation in Union queries", + "force apply AQE", + "tree string output", + "control a plan explain mode in listener vis SQLConf", + "AQE should set active session during execution", + "No deadlock in UI update", + "SPARK-35455: Unify empty relation optimization between normal and AQE optimizer - multi join" + ) + enableSuite[GlutenBinaryFileFormatSuite] + // Exception. + .exclude("column pruning - non-readable file") + enableSuite[GlutenCSVv1Suite] + .exclude("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeByPrefix("lineSep with 2 chars when multiLine set to") + enableSuite[GlutenCSVv2Suite] + .exclude("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeByPrefix("lineSep with 2 chars when multiLine set to") + .exclude("test for FAILFAST parsing mode") + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + enableSuite[GlutenCSVLegacyTimeParserSuite] + .exclude("SPARK-23786: warning should be printed if CSV header doesn't conform to schema") + .excludeByPrefix("lineSep with 2 chars when multiLine set to") + enableSuite[GlutenJsonV1Suite] + // FIXME: Array direct selection fails + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonV2Suite] + .exclude("SPARK-39731: Correctly parse dates and timestamps with yyyyMMdd pattern") + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenJsonLegacyTimeParserSuite] + .exclude("Complex field and type inferring") + .exclude("SPARK-4228 DataFrame to JSON") + enableSuite[GlutenValidateRequirementsSuite] + enableSuite[GlutenOrcColumnarBatchReaderSuite] + enableSuite[GlutenOrcFilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcPartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + enableSuite[GlutenOrcV1PartitionDiscoverySuite] + .exclude("read partitioned table - normal case") + .exclude("read partitioned table - with nulls") + .exclude("read partitioned table - partition key included in orc file") + .exclude("read partitioned table - with nulls and partition keys are included in Orc file") + enableSuite[GlutenOrcV1QuerySuite] + // Rewrite to disable Spark's columnar reader. + .exclude("Simple selection form ORC table") + .exclude("simple select queries") + .exclude("overwriting") + .exclude("self-join") + .exclude("columns only referenced by pushed down filters should remain") + .exclude("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when" + + " compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + enableSuite[GlutenOrcV2QuerySuite] + .exclude("Read/write binary data") + .exclude("Read/write all types with non-primitive type") + // Rewrite to disable Spark's columnar reader. + .exclude("Simple selection form ORC table") + .exclude("Creating case class RDD table") + .exclude("save and load case class RDD with `None`s as orc") + .exclude("SPARK-16610: Respect orc.compress (i.e., OrcConf.COMPRESS) when compression is unset") + .exclude("Compression options for writing to an ORC file (SNAPPY, ZLIB and NONE)") + .exclude("appending") + .exclude("nested data - struct with array field") + .exclude("nested data - array of struct") + .exclude("SPARK-9170: Don't implicitly lowercase of user-provided columns") + .exclude("SPARK-10623 Enable ORC PPD") + .exclude("SPARK-14962 Produce correct results on array type with isnotnull") + .exclude("SPARK-15198 Support for pushing down filters for boolean types") + .exclude("Support for pushing down filters for decimal types") + .exclude("Support for pushing down filters for timestamp types") + .exclude("column nullability and comment - write and then read") + .exclude("Empty schema does not read data from ORC file") + .exclude("read from multiple orc input paths") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") + .exclude("LZO compression options for writing to an ORC file") + .exclude("Schema discovery on empty ORC files") + .exclude("SPARK-21791 ORC should support column names with dot") + .exclude("SPARK-25579 ORC PPD should support column names with dot") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + .exclude("SPARK-37728: Reading nested columns with ORC vectorized reader should not") + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("Read/write all timestamp types") + .exclude("SPARK-37463: read/write Timestamp ntz to Orc with different time zone") + .exclude("SPARK-39381: Make vectorized orc columar writer batch size configurable") + .exclude("SPARK-39830: Reading ORC table that requires type promotion may throw AIOOBE") + .exclude("simple select queries") + .exclude("overwriting") + .exclude("self-join") + .exclude("columns only referenced by pushed down filters should remain") + .exclude("SPARK-5309 strings stored using dictionary compression in orc") + // For exception test. + .exclude("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") + enableSuite[GlutenOrcSourceSuite] + // Rewrite to disable Spark's columnar reader. + .exclude("SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("SPARK-31284: compatibility with Spark 2.4 in reading timestamps") + .exclude("SPARK-31284, SPARK-31423: rebasing timestamps in write") + .exclude("SPARK-34862: Support ORC vectorized reader for nested column") + // Ignored to disable vectorized reading check. + .exclude("SPARK-36594: ORC vectorized reader should properly check maximal number of fields") + .exclude("create temporary orc table") + .exclude("create temporary orc table as") + .exclude("appending insert") + .exclude("overwrite insert") + .exclude("SPARK-34897: Support reconcile schemas based on index after nested column pruning") + .exclude("Gluten - SPARK-31238: compatibility with Spark 2.4 in reading dates") + .exclude("Gluten - SPARK-31238, SPARK-31423: rebasing dates in write") + .exclude("Gluten - SPARK-34862: Support ORC vectorized reader for nested column") + // exclude as struct not supported + .exclude("SPARK-36663: OrcUtils.toCatalystSchema should correctly handle a column name which consists of only numbers") + .exclude("SPARK-37812: Reuse result row when deserializing a struct") + // rewrite + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=true)") + .exclude("SPARK-36931: Support reading and writing ANSI intervals (spark.sql.orc.enableVectorizedReader=true, spark.sql.orc.enableNestedColumnVectorizedReader=false)") + enableSuite[GlutenOrcV1FilterSuite] + .exclude("SPARK-32622: case sensitivity in predicate pushdown") + enableSuite[GlutenOrcV1SchemaPruningSuite] + .exclude( + "Spark vectorized reader - without partition data column - select only top-level fields") + .exclude("Spark vectorized reader - with partition data column - select only top-level fields") + .exclude("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + // Vectorized reading. + .exclude("Spark vectorized reader - without partition data column - " + + "select only expressions without references") + .exclude("Spark vectorized reader - with partition data column - " + + "select only expressions without references") + .exclude("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude( + "Spark vectorized reader - without partition data column - select a single complex field") + .exclude("Spark vectorized reader - with partition data column - select a single complex field") + .exclude( + "Non-vectorized reader - without partition data column - select a single complex field") + .exclude("Non-vectorized reader - with partition data column - select a single complex field") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - without partition data column - empty schema intersection") + .exclude("Spark vectorized reader - with partition data column - empty schema intersection") + .exclude("Non-vectorized reader - without partition data column - empty schema intersection") + .exclude("Non-vectorized reader - with partition data column - empty schema intersection") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function") + .exclude( + "Non-vectorized reader - with partition data column - select nested field in window function") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .exclude("Spark vectorized reader - with partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - without partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - with partition data column - select nested field in Sort") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - with partition data column - select nested field in Expand") + .exclude( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .exclude("Non-vectorized reader - with partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + enableSuite[GlutenOrcV2SchemaPruningSuite] + .exclude( + "Spark vectorized reader - without partition data column - select only top-level fields") + .exclude("Spark vectorized reader - with partition data column - select only top-level fields") + .exclude("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after join") + .exclude("Spark vectorized reader - " + + "without partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - " + + "with partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Spark vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - without partition data column - select a single complex field with disabled nested schema pruning") + .exclude("Non-vectorized reader - with partition data column - select a single complex field with disabled nested schema pruning") + .exclude( + "Spark vectorized reader - without partition data column - select a single complex field") + .exclude("Spark vectorized reader - with partition data column - select a single complex field") + .exclude( + "Non-vectorized reader - without partition data column - select a single complex field") + .exclude("Non-vectorized reader - with partition data column - select a single complex field") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and its parent struct") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and its parent struct") + .exclude("Spark vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - without partition data column - select a single complex field array and its parent struct array") + .exclude("Non-vectorized reader - with partition data column - select a single complex field array and its parent struct array") + .exclude("Spark vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - without partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Non-vectorized reader - with partition data column - select a single complex field from a map entry and its parent map entry") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and the partition column") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and the partition column") + .exclude("Spark vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Spark vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - without partition data column - partial schema intersection - select missing subfield") + .exclude("Non-vectorized reader - with partition data column - partial schema intersection - select missing subfield") + .exclude( + "Spark vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude( + "Non-vectorized reader - without partition data column - no unnecessary schema pruning") + .exclude("Non-vectorized reader - with partition data column - no unnecessary schema pruning") + .exclude("Spark vectorized reader - without partition data column - empty schema intersection") + .exclude("Spark vectorized reader - with partition data column - empty schema intersection") + .exclude("Non-vectorized reader - without partition data column - empty schema intersection") + .exclude("Non-vectorized reader - with partition data column - empty schema intersection") + .exclude("Spark vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - without partition data column - select a single complex field and is null expression in project") + .exclude("Non-vectorized reader - with partition data column - select a single complex field and is null expression in project") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map key using map_keys") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map key using map_keys") + .exclude("Spark vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - without partition data column - select nested field from a complex map value using map_values") + .exclude("Non-vectorized reader - with partition data column - select nested field from a complex map value using map_values") + .exclude("Spark vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Spark vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - with partition data column - select explode of nested field of array of struct") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after join") + .exclude("Non-vectorized reader - without partition data column - select one deep nested complex field after outer join") + .exclude("Non-vectorized reader - with partition data column - select one deep nested complex field after outer join") + .exclude("Spark vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - without partition data column - select nested field in aggregation function of Aggregate") + .exclude("Non-vectorized reader - with partition data column - select nested field in aggregation function of Aggregate") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function") + .exclude( + "Non-vectorized reader - with partition data column - select nested field in window function") + .exclude("Spark vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Spark vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - without partition data column - select nested field in window function and then order by") + .exclude("Non-vectorized reader - with partition data column - select nested field in window function and then order by") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Sort") + .exclude("Spark vectorized reader - with partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - without partition data column - select nested field in Sort") + .exclude("Non-vectorized reader - with partition data column - select nested field in Sort") + .exclude( + "Spark vectorized reader - without partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - with partition data column - select nested field in Expand") + .exclude( + "Non-vectorized reader - without partition data column - select nested field in Expand") + .exclude("Non-vectorized reader - with partition data column - select nested field in Expand") + .exclude("Spark vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - without partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Non-vectorized reader - with partition data column - SPARK-32163: nested pruning should work even with cosmetic variations") + .exclude("Spark vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Spark vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - without partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Non-vectorized reader - with partition data column - SPARK-38918: nested schema pruning with correlated subqueries") + .exclude("Case-sensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with exact column names") + .exclude("Case-insensitive parser - mixed-case schema - select with lowercase column names") + .exclude( + "Case-insensitive parser - mixed-case schema - select with different-case column names") + .exclude("Case-insensitive parser - mixed-case schema - subquery filter with different-case column names") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from array") + .exclude("Spark vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Spark vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - without partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("Non-vectorized reader - with partition data column - SPARK-34963: extract case-insensitive struct field from struct") + .exclude("SPARK-36352: Spark should check result plan's output schema name") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT EXISTS subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated IN subquery") + .exclude("Spark vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Spark vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - without partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + .exclude("Non-vectorized reader - with partition data column - SPARK-38977: schema pruning with correlated NOT IN subquery") + enableSuite[GlutenParquetColumnIndexSuite] + // Rewrite by just removing test timestamp. + .exclude("test reading unaligned pages - test all types") + enableSuite[GlutenParquetCompressionCodecPrecedenceSuite] + enableSuite[GlutenParquetDeltaByteArrayEncodingSuite] + enableSuite[GlutenParquetDeltaEncodingInteger] + enableSuite[GlutenParquetDeltaEncodingLong] + enableSuite[GlutenParquetDeltaLengthByteArrayEncodingSuite] + enableSuite[GlutenParquetEncodingSuite] + // exclude as cases use Vectorization Column reader + .exclude("parquet v2 pages - delta encoding") + .exclude("parquet v2 pages - rle encoding for boolean value columns") + enableSuite[GlutenParquetFieldIdIOSuite] + enableSuite[GlutenParquetFileFormatV1Suite] + // exclude for vectorization column reader + .exclude("support batch reads for schema") + enableSuite[GlutenParquetFileFormatV2Suite] + // exclude for vectorization column reader + .exclude("support batch reads for schema") + 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") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("filter pushdown - StringStartsWith") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .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") + .exclude("filter pushdown - StringPredicate") + 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") + .exclude("filter pushdown - date") + // Exception bebaviour. + .exclude("SPARK-25207: exception when duplicate fields in case-insensitive mode") + // Ignore Spark's filter pushdown check. + .exclude("Filters should be pushed down for vectorized Parquet reader at row group level") + .exclude("SPARK-31026: Parquet predicate pushdown for fields having dots in the names") + .exclude("Filters should be pushed down for Parquet readers at row group level") + .exclude("filter pushdown - StringStartsWith") + .exclude("SPARK-17091: Convert IN predicate to Parquet filter push-down") + .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") + .exclude("filter pushdown - StringPredicate") + .exclude("Gluten - filter pushdown - date") + enableSuite[GlutenParquetInteroperabilitySuite] + .exclude("parquet timestamp conversion") + enableSuite[GlutenParquetIOSuite] + // Disable Spark's vectorized reading tests. + .exclude("Standard mode - fixed-length decimals") + .exclude("Legacy mode - fixed-length decimals") + .exclude("SPARK-34167: read LongDecimals with precision < 10, VectorizedReader true") + .exclude("read dictionary encoded decimals written as FIXED_LEN_BYTE_ARRAY") + .exclude("read dictionary encoded decimals written as INT64") + .exclude("read dictionary encoded decimals written as INT32") + .exclude("SPARK-34817: Read UINT_64 as Decimal from parquet") + // Spark plans scan schema as (i16/i32/i64) so the fallback does not take effect. + // But Velox reads data based on the schema acquired from file metadata, + // while i8 is not supported, so error occurs. + .exclude("SPARK-34817: Read UINT_8/UINT_16/UINT_32 from parquet") + // Exception. + .exclude("SPARK-35640: read binary as timestamp should throw schema incompatible error") + // Exception msg. + .exclude("SPARK-35640: int as long should throw schema incompatible error") + // Timestamp is read as INT96. + .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") + .exclude("SPARK-36726: test incorrect Parquet row group file offset") + .exclude("SPARK-41096: FIXED_LEN_BYTE_ARRAY support") + .exclude("SPARK-40128 read DELTA_LENGTH_BYTE_ARRAY encoded strings") + .exclude("Read TimestampNTZ and TimestampLTZ for various logical TIMESTAMP types") + enableSuite[GlutenParquetV1PartitionDiscoverySuite] + // Timezone is not supported yet. + .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + // rewrite + .exclude("Various partition value types") + .exclude(("Various inferred partition value types")) + enableSuite[GlutenParquetV2PartitionDiscoverySuite] + // Timezone is not supported yet. + .exclude("Resolve type conflicts - decimals, dates and timestamps in partition column") + // rewrite + .exclude("Various partition value types") + .exclude(("Various inferred partition value types")) + enableSuite[GlutenParquetProtobufCompatibilitySuite] + enableSuite[GlutenParquetV1QuerySuite] + // Only for testing a type mismatch issue caused by hive (before hive 2.2). + // Only reproducible when spark.sql.parquet.enableVectorizedReader=true. + .exclude("SPARK-16632: read Parquet int32 as ByteType and ShortType") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("returning batch for wide table") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Timestamp is read as INT96. + .exclude("SPARK-10634 timestamp written and read as INT64 - truncation") + .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") + .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") + .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") + // new added in spark-3.3 and need fix later, random failure may caused by memory free + .exclude("SPARK-39833: pushed filters with project without filter columns") + .exclude("SPARK-39833: pushed filters with count()") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV2QuerySuite] + // Only for testing a type mismatch issue caused by hive (before hive 2.2). + // Only reproducible when spark.sql.parquet.enableVectorizedReader=true. + .exclude("SPARK-16632: read Parquet int32 as ByteType and ShortType") + .exclude("Enabling/disabling ignoreCorruptFiles") + .exclude("returning batch for wide table") + // decimal failed ut + .exclude("SPARK-34212 Parquet should read decimals correctly") + // Timestamp is read as INT96. + .exclude("SPARK-10634 timestamp written and read as INT64 - truncation") + .exclude("Migration from INT96 to TIMESTAMP_MICROS timestamp type") + .exclude("SPARK-10365 timestamp written and read as INT64 - TIMESTAMP_MICROS") + .exclude("SPARK-36182: read TimestampNTZ as TimestampLTZ") + // Rewrite because the filter after datasource is not needed. + .exclude( + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") + enableSuite[GlutenParquetV1SchemaPruningSuite] + enableSuite[GlutenParquetV2SchemaPruningSuite] + enableSuite[GlutenParquetRebaseDatetimeV1Suite] + // jar path and ignore PARQUET_REBASE_MODE_IN_READ, rewrite some + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetRebaseDatetimeV2Suite] + // jar path and ignore PARQUET_REBASE_MODE_IN_READ + .excludeByPrefix("SPARK-31159") + .excludeByPrefix("SPARK-35427") + enableSuite[GlutenParquetSchemaInferenceSuite] + enableSuite[GlutenParquetSchemaSuite] + // error message mismatch is accepted + .exclude("schema mismatch failure error message for parquet reader") + .exclude("schema mismatch failure error message for parquet vectorized reader") + .excludeByPrefix("SPARK-40819:") + enableSuite[GlutenParquetThriftCompatibilitySuite] + // Rewrite for file locating. + .exclude("Read Parquet file generated by parquet-thrift") + enableSuite[GlutenParquetVectorizedSuite] + enableSuite[GlutenTextV1Suite] + enableSuite[GlutenTextV2Suite] + enableSuite[GlutenDataSourceV2StrategySuite] + enableSuite[GlutenFileTableSuite] + enableSuite[GlutenV2PredicateSuite] + enableSuite[GlutenBucketingUtilsSuite] + enableSuite[GlutenDataSourceStrategySuite] + enableSuite[GlutenDataSourceSuite] + enableSuite[GlutenFileFormatWriterSuite] + .excludeByPrefix("empty file should be skipped while write to file") + enableSuite[GlutenFileIndexSuite] + enableSuite[GlutenFileMetadataStructSuite] + .exclude("SPARK-41896: Filter on row_index and a stored column at the same time") + .exclude("SPARK-43450: Filter on aliased _metadata.row_index") + enableSuite[GlutenParquetV1AggregatePushDownSuite] + enableSuite[GlutenParquetV2AggregatePushDownSuite] + enableSuite[GlutenOrcV1AggregatePushDownSuite] + enableSuite[GlutenOrcV2AggregatePushDownSuite] + enableSuite[GlutenParquetCodecSuite] + // Unsupported compression codec. + .exclude("write and read - file source parquet - codec: lz4") + enableSuite[GlutenOrcCodecSuite] + enableSuite[GlutenFileSourceStrategySuite] + // Plan comparison. + .exclude("partitioned table - after scan filters") + enableSuite[GlutenHadoopFileLinesReaderSuite] + enableSuite[GlutenPathFilterStrategySuite] + enableSuite[GlutenPathFilterSuite] + enableSuite[GlutenPruneFileSourcePartitionsSuite] + enableSuite[GlutenCSVReadSchemaSuite] + enableSuite[GlutenHeaderCSVReadSchemaSuite] + enableSuite[GlutenJsonReadSchemaSuite] + enableSuite[GlutenOrcReadSchemaSuite] + .exclude("append column into middle") + .exclude("hide column in the middle") + .exclude("change column position") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("read as string") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + .exclude("change column type from float to decimal") + .exclude("change column type from double to decimal") + .exclude("read float, double, decimal together") + .exclude("add a nested column at the end of the leaf struct column") + .exclude("add a nested column in the middle of the leaf struct column") + .exclude("add a nested column at the end of the middle struct column") + .exclude("add a nested column in the middle of the middle struct column") + .exclude("hide a nested column at the end of the leaf struct column") + .exclude("hide a nested column in the middle of the leaf struct column") + .exclude("hide a nested column at the end of the middle struct column") + .exclude("hide a nested column in the middle of the middle struct column") + enableSuite[GlutenVectorizedOrcReadSchemaSuite] + // Rewrite to disable Spark's vectorized reading. + .exclude("change column position") + .exclude("read byte, int, short, long together") + .exclude("read float and double together") + .exclude("append column into middle") + .exclude("add a nested column at the end of the leaf struct column") + .exclude("add a nested column in the middle of the leaf struct column") + .exclude("add a nested column at the end of the middle struct column") + .exclude("add a nested column in the middle of the middle struct column") + .exclude("hide a nested column at the end of the leaf struct column") + .exclude("hide a nested column in the middle of the leaf struct column") + .exclude("hide a nested column at the end of the middle struct column") + .exclude("hide a nested column in the middle of the middle struct column") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("change column type from float to double") + .exclude("Gluten - read byte, int, short, long together") + .exclude("Gluten - read float and double together") + enableSuite[GlutenMergedOrcReadSchemaSuite] + .exclude("append column into middle") + .exclude("add a nested column at the end of the leaf struct column") + .exclude("add a nested column in the middle of the leaf struct column") + .exclude("add a nested column at the end of the middle struct column") + .exclude("add a nested column in the middle of the middle struct column") + .exclude("hide a nested column at the end of the leaf struct column") + .exclude("hide a nested column in the middle of the leaf struct column") + .exclude("hide a nested column at the end of the middle struct column") + .exclude("hide a nested column in the middle of the middle struct column") + .exclude("change column type from boolean to byte/short/int/long") + .exclude("change column type from byte to short/int/long") + .exclude("change column type from short to int/long") + .exclude("change column type from int to long") + .exclude("read byte, int, short, long together") + .exclude("change column type from float to double") + .exclude("read float and double together") + enableSuite[GlutenParquetReadSchemaSuite] + enableSuite[GlutenVectorizedParquetReadSchemaSuite] + enableSuite[GlutenMergedParquetReadSchemaSuite] + enableSuite[GlutenEnsureRequirementsSuite] + // FIXME: yan + .exclude("reorder should handle PartitioningCollection") + // Rewrite to change the shuffle partitions for optimizing repartition + .excludeByPrefix("SPARK-35675") + .exclude("SPARK-41986: Introduce shuffle on SinglePartition") + +// enableSuite[GlutenBroadcastJoinSuite] +// .exclude("Shouldn't change broadcast join buildSide if user clearly specified") +// .exclude("Shouldn't bias towards build right if user didn't specify") +// .exclude("SPARK-23192: broadcast hint should be retained after using the cached data") +// .exclude("broadcast hint isn't propagated after a join") +// .exclude("broadcast join where streamed side's output partitioning is HashPartitioning") + + enableSuite[GlutenExistenceJoinSuite] + enableSuite[GlutenInnerJoinSuite] + enableSuite[GlutenOuterJoinSuite] + enableSuite[FallbackStrategiesSuite] + enableSuite[GlutenBroadcastExchangeSuite] + enableSuite[GlutenCoalesceShufflePartitionsSuite] + // FIXME: yan + .exclude("determining the number of reducers: aggregate operator") + .exclude("determining the number of reducers: join operator") + .exclude("determining the number of reducers: complex query 1") + .exclude("determining the number of reducers: complex query 2") + .exclude("Gluten - determining the number of reducers: aggregate operator") + .exclude("Gluten - determining the number of reducers: join operator") + .exclude("Gluten - determining the number of reducers: complex query 1") + .exclude("Gluten - determining the number of reducers: complex query 2") + .exclude("Union two datasets with different pre-shuffle partition number") + enableSuite[GlutenExchangeSuite] + // ColumnarShuffleExchangeExec does not support doExecute() method + .exclude("shuffling UnsafeRows in exchange") + // ColumnarShuffleExchangeExec does not support SORT_BEFORE_REPARTITION + .exclude("SPARK-23207: Make repartition() generate consistent output") + // This test will re-run in GlutenExchangeSuite with shuffle partitions > 1 + .exclude("Exchange reuse across the whole plan") + enableSuite[GlutenReplaceHashWithSortAggSuite] + .exclude("replace partial hash aggregate with sort aggregate") + .exclude("replace partial and final hash aggregate together with sort aggregate") + .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") + enableSuite[GlutenReuseExchangeAndSubquerySuite] + enableSuite[GlutenSameResultSuite] + enableSuite[GlutenSortSuite] + // spill not supported yet. + enableSuite[GlutenSQLWindowFunctionSuite].exclude("test with low buffer spill threshold") + enableSuite[GlutenTakeOrderedAndProjectSuite] + enableSuite[GlutenSessionExtensionSuite] + enableSuite[TestFileSourceScanExecTransformer] + enableSuite[GlutenBucketedReadWithoutHiveSupportSuite] + // Exclude the following suite for plan changed from SMJ to SHJ. + .exclude("avoid shuffle when join 2 bucketed tables") + .exclude("avoid shuffle and sort when sort columns are a super set of join keys") + .exclude("only shuffle one side when join bucketed table and non-bucketed table") + .exclude("only shuffle one side when 2 bucketed tables have different bucket number") + .exclude("only shuffle one side when 2 bucketed tables have different bucket keys") + .exclude("shuffle when join keys are not equal to bucket keys") + .exclude("shuffle when join 2 bucketed tables with bucketing disabled") + .exclude("check sort and shuffle when bucket and sort columns are join keys") + .exclude("only sort one side when sort columns are different") + .exclude("only sort one side when sort columns are same but their ordering is different") + .exclude("SPARK-17698 Join predicates should not contain filter clauses") + .exclude("SPARK-19122 Re-order join predicates if they match with the child's" + + " output partitioning") + .exclude("SPARK-19122 No re-ordering should happen if set of join columns != set of child's " + + "partitioning columns") + .exclude("SPARK-29655 Read bucketed tables obeys spark.sql.shuffle.partitions") + .exclude("SPARK-32767 Bucket join should work if SHUFFLE_PARTITIONS larger than bucket number") + .exclude("bucket coalescing eliminates shuffle") + .exclude("bucket coalescing is not satisfied") + .excludeByPrefix("bucket coalescing is applied when join expressions match") + enableSuite[GlutenBucketedWriteWithoutHiveSupportSuite] + enableSuite[GlutenCreateTableAsSelectSuite] + // TODO Gluten can not catch the spark exception in Driver side. + .exclude("CREATE TABLE USING AS SELECT based on the file without write permission") + .exclude("create a table, drop it and create another one with the same name") + enableSuite[GlutenDDLSourceLoadSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite] + enableSuite[GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE] + enableSuite[GlutenExternalCommandRunnerSuite] + enableSuite[GlutenFilteredScanSuite] + enableSuite[GlutenFiltersSuite] + enableSuite[GlutenInsertSuite] + .exclude("INSERT rows, ALTER TABLE ADD COLUMNS with DEFAULTs, then SELECT them") + .exclude("SPARK-39557 INSERT INTO statements with tables with array defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with struct defaults") + .exclude("SPARK-39557 INSERT INTO statements with tables with map defaults") + .exclude("SPARK-39844 Restrict adding DEFAULT columns for existing tables to certain sources") + enableSuite[GlutenPartitionedWriteSuite] + enableSuite[GlutenPathOptionSuite] + enableSuite[GlutenPrunedScanSuite] + enableSuite[GlutenResolvedDataSourceSuite] + enableSuite[GlutenSaveLoadSuite] + enableSuite[GlutenTableScanSuite] + enableSuite[GlutenApproxCountDistinctForIntervalsQuerySuite] + enableSuite[GlutenApproximatePercentileQuerySuite] + // requires resource files from Vanilla spark jar + .exclude("SPARK-32908: maximum target error in percentile_approx") + enableSuite[GlutenCachedTableSuite] + .exclude("InMemoryRelation statistics") + // Extra ColumnarToRow is needed to transform vanilla columnar data to gluten columnar data. + .exclude("SPARK-37369: Avoid redundant ColumnarToRow transition on InMemoryTableScan") + enableSuite[GlutenFileSourceCharVarcharTestSuite] + enableSuite[GlutenDSV2CharVarcharTestSuite] + enableSuite[GlutenColumnExpressionSuite] + enableSuite[GlutenComplexTypeSuite] + enableSuite[GlutenConfigBehaviorSuite] + // Will be fixed by cleaning up ColumnarShuffleExchangeExec. + .exclude("SPARK-22160 spark.sql.execution.rangeExchange.sampleSizePerPartition") + enableSuite[GlutenCountMinSketchAggQuerySuite] + enableSuite[GlutenCsvFunctionsSuite] + enableSuite[GlutenCTEHintSuite] + .exclude("Resolve join hint in CTE") + enableSuite[GlutenCTEInlineSuiteAEOff] + enableSuite[GlutenCTEInlineSuiteAEOn] + enableSuite[GlutenDataFrameAggregateSuite] + .exclude( + "zero moments", // [velox does not return NaN] + "SPARK-26021: NaN and -0.0 in grouping expressions", // NaN case + // incorrect result, distinct NaN case + "SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate" + ) + enableSuite[GlutenDataFrameAsOfJoinSuite] + enableSuite[GlutenDataFrameComplexTypeSuite] + enableSuite[GlutenDataFrameFunctionsSuite] + // blocked by Velox-5768 + .exclude("aggregate function - array for primitive type containing null") + .exclude("aggregate function - array for non-primitive type") + enableSuite[GlutenDataFrameHintSuite] + enableSuite[GlutenDataFrameImplicitsSuite] + enableSuite[GlutenDataFrameJoinSuite] + enableSuite[GlutenDataFrameNaFunctionsSuite] + .exclude( + // NaN case + "replace nan with float", + "replace nan with double" + ) + enableSuite[GlutenDataFramePivotSuite] + // substring issue + .exclude("pivot with column definition in groupby") + enableSuite[GlutenDataFrameRangeSuite] + enableSuite[GlutenDataFrameSelfJoinSuite] + enableSuite[GlutenDataFrameSessionWindowingSuite] + enableSuite[GlutenDataFrameSetOperationsSuite] + // exclude as map not supported + .exclude("SPARK-36797: Union should resolve nested columns as top-level columns") + .exclude("SPARK-37371: UnionExec should support columnar if all children support columnar") + enableSuite[GlutenDataFrameStatSuite] + enableSuite[GlutenDataFrameSuite] + // Rewrite these tests because it checks Spark's physical operators. + .excludeByPrefix("SPARK-22520", "reuse exchange") + .exclude( + /** + * Rewrite these tests because the rdd partition is equal to the configuration + * "spark.sql.shuffle.partitions". + */ + "repartitionByRange", + "distributeBy and localSort", + // Mismatch when max NaN and infinite value + "NaN is greater than all other non-NaN numeric values", + // Rewrite this test because the describe functions creates unmatched plan. + "describe", + // The describe issue is just fixed by https://github.com/apache/spark/pull/40914. + // We can enable the below test for spark 3.4 and higher versions. + "Gluten - describe", + // decimal failed ut. + "SPARK-22271: mean overflows and returns null for some decimal variables", + // Not supported for approx_count_distinct + "SPARK-34165: Add count_distinct to summary", + "SPARK-41048: Improve output partitioning and ordering with AQE cache" + ) + enableSuite[GlutenDataFrameTimeWindowingSuite] + enableSuite[GlutenDataFrameTungstenSuite] + enableSuite[GlutenDataFrameWindowFramesSuite] + // Local window fixes are not added. + .exclude("range between should accept int/long values as boundary") + .exclude("unbounded preceding/following range between with aggregation") + .exclude("sliding range between with aggregation") + .exclude("store and retrieve column stats in different time zones") + .exclude("rows between should accept int/long values as boundary") + enableSuite[GlutenDataFrameWriterV2Suite] + enableSuite[GlutenDatasetAggregatorSuite] + enableSuite[GlutenDatasetCacheSuite] + enableSuite[GlutenDatasetOptimizationSuite] + enableSuite[GlutenDatasetPrimitiveSuite] + enableSuite[GlutenDatasetSerializerRegistratorSuite] + enableSuite[GlutenDatasetSuite] + // Rewrite the following two tests in GlutenDatasetSuite. + .exclude("dropDuplicates: columns with same column name") + .exclude("groupBy.as") + // Map could not contain non-scalar type. + .exclude("as map of case class - reorder fields by name") + // exclude as velox has different behavior in these cases + .exclude("SPARK-40407: repartition should not result in severe data skew") + .exclude("SPARK-40660: Switch to XORShiftRandom to distribute elements") + enableSuite[GlutenDateFunctionsSuite] + // The below two are replaced by two modified versions. + .exclude("unix_timestamp") + .exclude("to_unix_timestamp") + enableSuite[GlutenDeprecatedAPISuite] + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOff] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + .excludeByPrefix("static scan metrics") + .excludeByPrefix("Gluten - static scan metrics") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOn] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOff] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOn] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan] + .exclude("SPARK-38674: Remove useless deduplicate in SubqueryBroadcastExec") + enableSuite[GlutenExpressionsSchemaSuite] + .exclude("Check schemas for expression examples") + enableSuite[GlutenExtraStrategiesSuite] + enableSuite[GlutenFileBasedDataSourceSuite] + // test data path is jar path, rewrite + .exclude("Option recursiveFileLookup: disable partition inferring") + // gluten executor exception cannot get in driver, rewrite + .exclude("Spark native readers should respect spark.sql.caseSensitive - parquet") + // shuffle_partitions config is different, rewrite + .excludeByPrefix("SPARK-22790") + // plan is different cause metric is different, rewrite + .excludeByPrefix("SPARK-25237") + // ignoreMissingFiles mode, wait to fix + .exclude("Enabling/disabling ignoreMissingFiles using parquet") + .exclude("Enabling/disabling ignoreMissingFiles using orc") + .exclude("Spark native readers should respect spark.sql.caseSensitive - orc") + .exclude("Return correct results when data columns overlap with partition columns") + .exclude("Return correct results when data columns overlap with partition " + + "columns (nested data)") + .exclude("SPARK-31116: Select nested schema with case insensitive mode") + // exclude as original metric not correct when task offloaded to velox + .exclude("SPARK-37585: test input metrics for DSV2 with output limits") + .exclude("SPARK-23271 empty RDD when saved should write a metadata only file - orc") + .exclude("SPARK-22146 read files containing special characters using orc") + .exclude("SPARK-30362: test input metrics for DSV2") + .exclude("Do not use cache on overwrite") + .exclude("Do not use cache on append") + .exclude("File source v2: support partition pruning") + .exclude("File source v2: support passing data filters to FileScan without partitionFilters") + enableSuite[GlutenFileScanSuite] + enableSuite[GlutenGeneratorFunctionSuite] + enableSuite[GlutenInjectRuntimeFilterSuite] + // FIXME: yan + .exclude("Merge runtime bloom filters") + enableSuite[GlutenIntervalFunctionsSuite] + enableSuite[GlutenJoinSuite] + // exclude as it check spark plan + .exclude("SPARK-36794: Ignore duplicated key when building relation for semi/anti hash join") + .exclude( + "SPARK-43113: Full outer join with duplicate stream-side references in condition (SMJ)") + enableSuite[GlutenMathFunctionsSuite] + enableSuite[GlutenMetadataCacheSuite] + .exclude("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") + enableSuite[GlutenMiscFunctionsSuite] + enableSuite[GlutenNestedDataSourceV1Suite] + enableSuite[GlutenNestedDataSourceV2Suite] + enableSuite[GlutenProcessingTimeSuite] + enableSuite[GlutenProductAggSuite] + enableSuite[GlutenReplaceNullWithFalseInPredicateEndToEndSuite] + enableSuite[GlutenScalaReflectionRelationSuite] + enableSuite[GlutenSerializationSuite] + // following UT is removed in spark3.3.1 + // enableSuite[GlutenSimpleShowCreateTableSuite] + enableSuite[GlutenFileSourceSQLInsertTestSuite] + .exclude( + "SPARK-41982: treat the partition field as string literal when keepPartitionSpecAsStringLiteral is enabled") + enableSuite[GlutenDSV2SQLInsertTestSuite] + .exclude( + "SPARK-41982: treat the partition field as string literal when keepPartitionSpecAsStringLiteral is enabled") + enableSuite[GlutenSQLQuerySuite] + // Decimal precision exceeds. + .exclude("should be able to resolve a persistent view") + // Unstable. Needs to be fixed. + .exclude("SPARK-36093: RemoveRedundantAliases should not change expression's name") + // Rewrite from ORC scan to Parquet scan because ORC is not well supported. + .exclude("SPARK-28156: self-join should not miss cached view") + .exclude("SPARK-33338: GROUP BY using literal map should not fail") + // Rewrite to disable plan check for SMJ because SHJ is preferred in Gluten. + .exclude("SPARK-11111 null-safe join should not use cartesian product") + // Rewrite to change the information of a caught exception. + .exclude("SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") + // Different exception. + .exclude("run sql directly on files") + // Columnar shuffle cannot generate the expected number of partitions if the row of a input + // batch is less than the expected number of partitions. + .exclude("SPARK-24940: coalesce and repartition hint") + // Not useful and time consuming. + .exclude("SPARK-33084: Add jar support Ivy URI in SQL") + .exclude("SPARK-33084: Add jar support Ivy URI in SQL -- jar contains udf class") + .exclude("SPARK-38548: try_sum should return null if overflow happens before merging") + .exclude("the escape character is not allowed to end with") + .exclude("SPARK-40245: Fix FileScan canonicalization when partition or data filter columns are not read") + enableSuite[GlutenSQLQueryTestSuite] + enableSuite[GlutenStatisticsCollectionSuite] + .exclude("SPARK-33687: analyze all tables in a specific database") + enableSuite[GlutenSubquerySuite] + .excludeByPrefix( + "SPARK-26893" // Rewrite this test because it checks Spark's physical operators. + ) + // exclude as it checks spark plan + .exclude("SPARK-36280: Remove redundant aliases after RewritePredicateSubquery") + enableSuite[GlutenTypedImperativeAggregateSuite] + enableSuite[GlutenUnwrapCastInComparisonEndToEndSuite] + // Rewrite with NaN test cases excluded. + .exclude("cases when literal is max") + enableSuite[GlutenXPathFunctionsSuite] + enableSuite[GlutenFallbackSuite] + enableSuite[GlutenHiveSQLQuerySuite] +} +// scalastyle:on line.size.limit diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala new file mode 100644 index 000000000000..86ef1238965f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproxCountDistinctForIntervalsQuerySuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenApproxCountDistinctForIntervalsQuerySuite + extends ApproxCountDistinctForIntervalsQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIteratorV2.java b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala similarity index 74% rename from gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIteratorV2.java rename to gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala index 84de3cd15279..eb82baa78dac 100644 --- a/gluten-core/src/main/java/io/glutenproject/vectorized/GeneralInIteratorV2.java +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenApproximatePercentileQuerySuite.scala @@ -14,14 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.glutenproject.vectorized; +package org.apache.spark.sql -import org.apache.spark.sql.vectorized.ColumnarBatch; +class GlutenApproximatePercentileQuerySuite + extends ApproximatePercentileQuerySuite + with GlutenSQLTestsTrait { -import java.util.Iterator; - -public abstract class GeneralInIteratorV2 extends GeneralInIterator { - public GeneralInIteratorV2(Iterator delegated) { - super(delegated); + override def testFile(fileName: String): String = { + Thread.currentThread().getContextClassLoader.getResource(fileName).toString } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala new file mode 100644 index 000000000000..744271e53a78 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenBloomFilterAggregateQuerySuite.scala @@ -0,0 +1,66 @@ +/* + * 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 io.glutenproject.GlutenConfig + +import org.apache.spark.sql.internal.SQLConf + +class GlutenBloomFilterAggregateQuerySuite + extends BloomFilterAggregateQuerySuite + with GlutenSQLTestsTrait { + import testImplicits._ + + test("Test bloom_filter_agg with big RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS") { + val table = "bloom_filter_test" + withSQLConf( + SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS.key -> "5000000", + GlutenConfig.COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS.key -> "4194304" + ) { + val numEstimatedItems = 5000000L + val numBits = GlutenConfig.getConf.veloxBloomFilterMaxNumBits + val sqlString = s""" + |SELECT every(might_contain( + | (SELECT bloom_filter_agg(col, + | cast($numEstimatedItems as long), + | cast($numBits as long)) + | FROM $table), + | col)) positive_membership_test + |FROM $table + """.stripMargin + withTempView(table) { + (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 200000L)) + .toDF("col") + .createOrReplaceTempView(table) + checkAnswer(spark.sql(sqlString), Row(true)) + } + } + } + + test("Test that might_contain on bloom_filter_agg with empty input") { + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), cast(123 as long))""".stripMargin), + Row(null) + ) + + checkAnswer( + spark.sql("""SELECT might_contain((select bloom_filter_agg(cast(id as long)) + | from range(1, 1)), null)""".stripMargin), + Row(null)) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala new file mode 100644 index 000000000000..8005bffc310d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEHintSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenCTEHintSuite extends CTEHintSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala new file mode 100644 index 000000000000..f5bdb254b2f3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCTEInlineSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenCTEInlineSuiteAEOff extends CTEInlineSuiteAEOff with GlutenSQLTestsTrait + +class GlutenCTEInlineSuiteAEOn extends CTEInlineSuiteAEOn with GlutenSQLTestsTrait diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala new file mode 100644 index 000000000000..d6a04713bcd9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCachedTableSuite.scala @@ -0,0 +1,43 @@ +/* + * 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 io.glutenproject.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.columnar.InMemoryRelation + +class GlutenCachedTableSuite + extends CachedTableSuite + with GlutenSQLTestsTrait + with AdaptiveSparkPlanHelper { + // for temporarily disable the columnar table cache globally. + sys.props.put(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.sql.shuffle.partitions", "5") + super.sparkConf.set(GlutenConfig.COLUMNAR_TABLE_CACHE_ENABLED.key, "true") + } + + test("GLUTEN - InMemoryRelation statistics") { + sql("CACHE TABLE testData") + spark.table("testData").queryExecution.withCachedData.collect { + case cached: InMemoryRelation => + assert(cached.stats.sizeInBytes === 1132) + } + } +} 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 new file mode 100644 index 000000000000..84502ace5110 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCharVarcharTestSuite.scala @@ -0,0 +1,23 @@ +/* + * 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 + +class GlutenFileSourceCharVarcharTestSuite + extends FileSourceCharVarcharTestSuite + with GlutenSQLTestsTrait {} + +class GlutenDSV2CharVarcharTestSuite extends DSV2CharVarcharTestSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala new file mode 100644 index 000000000000..edd2a5a9672d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenColumnExpressionSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenColumnExpressionSuite extends ColumnExpressionSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala new file mode 100644 index 000000000000..05f9a46d502e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenComplexTypesSuite.scala @@ -0,0 +1,97 @@ +/* + * 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 + +class GlutenComplexTypesSuite extends ComplexTypesSuite with GlutenSQLTestsTrait { + + override def beforeAll(): Unit = { + super.beforeAll() + spark + .range(10) + .selectExpr( + "(id % 2 = 0) as bool", + "cast(id as BYTE) as i8", + "cast(id as SHORT) as i16", + "cast(id as FLOAT) as fp32", + "cast(id as DOUBLE) as fp64", + "cast(id as DECIMAL(4, 2)) as dec", + "cast(cast(id as BYTE) as BINARY) as vbin", + "binary(id) as vbin1", + "map_from_arrays(array(id),array(id+2)) as map", + "array(id, id+1, id+2) as list", + "struct(cast(id as LONG) as a, cast(id+1 as STRING) as b) as struct" + ) + .write + .saveAsTable("tab_types") + } + + override def afterAll(): Unit = { + try { + spark.sql("DROP TABLE IF EXISTS tab_types") + } finally { + super.afterAll() + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "types bool/byte/short/float/double/decimal/binary/map/array/struct") { + val df = spark + .table("tab_types") + .selectExpr( + "bool", + "i8", + "i16", + "fp32", + "fp64", + "dec", + "vbin", + "length(vbin)", + "vbin1", + "length(vbin1)", + "struct", + "struct.a", + "list", + "map" + ) + .sort("i8") + .limit(1) + + checkAnswer( + df, + Seq( + Row( + true, + 0.toByte, + 0.toShort, + 0.toFloat, + 0.toDouble, + BigDecimal(0), + Array.fill[Byte](1)(0.toByte), + 1.toInt, + Array.fill[Byte](8)(0.toByte), + 8.toInt, + Row(0.toLong, "1"), + 0.toLong, + Array(0, 1, 2), + Map(0 -> 2) + )) + ) + + checkNamedStruct(df.queryExecution.optimizedPlan, expectedCount = 0) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala new file mode 100644 index 000000000000..c1984a5e22dd --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenConfigBehaviorSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenConfigBehaviorSuite extends ConfigBehaviorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala new file mode 100644 index 000000000000..182464c0a5ee --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCountMinSketchAggQuerySuite.scala @@ -0,0 +1,22 @@ +/* + * 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 + +/** End-to-end test suite for count_min_sketch. */ +class GlutenCountMinSketchAggQuerySuite + extends CountMinSketchAggQuerySuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala new file mode 100644 index 000000000000..0550fef442ff --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenCsvFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenCsvFunctionsSuite extends CsvFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala new file mode 100644 index 000000000000..c50792b94c4a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAggregateSuite.scala @@ -0,0 +1,225 @@ +/* + * 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 io.glutenproject.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.execution.aggregate.SortAggregateExec +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.functions._ + +import java.lang.{Long => JLong} + +class GlutenDataFrameAggregateSuite extends DataFrameAggregateSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + // blackTestNameList is defined in ClickHouseNotSupport + + test(GlutenTestConstants.GLUTEN_TEST + "count") { + // agg with no input col + assert(testData2.count() === testData2.rdd.map(_ => 1).count()) + + checkAnswer( + testData2.agg(count($"a"), sum_distinct($"a")), // non-partial + Row(6, 6.0)) + } + + test(GlutenTestConstants.GLUTEN_TEST + "null count") { + checkAnswer(testData3.groupBy($"a").agg(count($"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer(testData3.groupBy($"a").agg(count($"a" + $"b")), Seq(Row(1, 0), Row(2, 1))) + + checkAnswer( + testData3 + .agg(count($"a"), count($"b"), count(lit(1)), count_distinct($"a"), count_distinct($"b")), + Row(2, 1, 2, 2, 1)) + + // [wishlist] does not support sum distinct +// checkAnswer( +// testData3.agg(count($"b"), count_distinct($"b"), sum_distinct($"b")), // non-partial +// Row(1, 1, 2) +// ) + } + + test(GlutenTestConstants.GLUTEN_TEST + "groupBy") { + checkAnswer(testData2.groupBy("a").agg(sum($"b")), Seq(Row(1, 3), Row(2, 3), Row(3, 3))) + checkAnswer(testData2.groupBy("a").agg(sum($"b").as("totB")).agg(sum($"totB")), Row(9)) + checkAnswer(testData2.groupBy("a").agg(count("*")), Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("*" -> "count")), + Row(1, 2) :: Row(2, 2) :: Row(3, 2) :: Nil) + checkAnswer( + testData2.groupBy("a").agg(Map("b" -> "sum")), + Row(1, 3) :: Row(2, 3) :: Row(3, 3) :: Nil) + + val df1 = Seq(("a", 1, 0, "b"), ("b", 2, 4, "c"), ("a", 2, 3, "d")) + .toDF("key", "value1", "value2", "rest") + + checkAnswer(df1.groupBy("key").min(), df1.groupBy("key").min("value1", "value2").collect()) + checkAnswer(df1.groupBy("key").min("value2"), Seq(Row("a", 0), Row("b", 4))) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(3)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3))) +// ) +// +// val decimalDataWithNulls = spark.sparkContext.parallelize( +// DecimalData(1, 1) :: +// DecimalData(1, null) :: +// DecimalData(2, 1) :: +// DecimalData(2, null) :: +// DecimalData(3, 1) :: +// DecimalData(3, 2) :: +// DecimalData(null, 2) :: Nil).toDF() +// checkAnswer( +// decimalDataWithNulls.groupBy("a").agg(sum("b")), +// Seq(Row(new java.math.BigDecimal(1), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(1)), +// Row(new java.math.BigDecimal(3), new java.math.BigDecimal(3)), +// Row(null, new java.math.BigDecimal(2))) +// ) + } + + test(GlutenTestConstants.GLUTEN_TEST + "average") { + + checkAnswer(testData2.agg(avg($"a"), mean($"a")), Row(2.0, 2.0)) + + checkAnswer( + testData2.agg(avg($"a"), sum_distinct($"a")), // non-partial and test deprecated version + Row(2.0, 6.0) :: Nil) + + // [wishlist] does not support decimal +// checkAnswer( +// decimalData.agg(avg($"a")), +// Row(new java.math.BigDecimal(2))) +// +// checkAnswer( +// decimalData.agg(avg($"a"), sum_distinct($"a")), // non-partial +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) +// +// checkAnswer( +// decimalData.agg(avg($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2))) +// // non-partial +// checkAnswer( +// decimalData.agg( +// avg($"a" cast DecimalType(10, 2)), sum_distinct($"a" cast DecimalType(10, 2))), +// Row(new java.math.BigDecimal(2), new java.math.BigDecimal(6)) :: Nil) + } + + ignore("gluten SPARK-32038: NormalizeFloatingNumbers should work on distinct aggregate") { + withTempView("view") { + Seq( + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("mithunr", Float.NaN), + ("abellina", 1.0f), + ("abellina", 2.0f)).toDF("uid", "score").createOrReplaceTempView("view") + + val df = spark.sql("select uid, count(distinct score) from view group by 1 order by 1 asc") + checkAnswer(df, Row("abellina", 2) :: Row("mithunr", 1) :: Nil) + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "variance") { + checkAnswer( + testData2.agg(var_samp($"a"), var_pop($"a"), variance($"a")), + Row(0.8, 2.0 / 3.0, 0.8)) + checkAnswer(testData2.agg(var_samp("a"), var_pop("a"), variance("a")), Row(0.8, 2.0 / 3.0, 0.8)) + } + + test("aggregation with filter") { + Seq( + ("mithunr", 12.3f, 5.0f, true, 9.4f), + ("mithunr", 15.5f, 4.0f, false, 19.9f), + ("mithunr", 19.8f, 3.0f, false, 35.6f), + ("abellina", 20.1f, 2.0f, true, 98.0f), + ("abellina", 20.1f, 1.0f, true, 0.5f), + ("abellina", 23.6f, 2.0f, true, 3.9f) + ) + .toDF("uid", "time", "score", "pass", "rate") + .createOrReplaceTempView("view") + var df = spark.sql("select count(score) filter (where pass) from view group by time") + checkAnswer(df, Row(1) :: Row(0) :: Row(0) :: Row(2) :: Row(1) :: Nil) + + df = spark.sql("select count(score) filter (where pass) from view") + checkAnswer(df, Row(4) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view group by time") + checkAnswer(df, Row(0) :: Row(0) :: Row(1) :: Row(1) :: Row(0) :: Nil) + + df = spark.sql("select count(score) filter (where rate > 20) from view") + checkAnswer(df, Row(2) :: Nil) + } + + test(GlutenTestConstants.GLUTEN_TEST + "extend with cast expression") { + checkAnswer( + decimalData.agg( + sum($"a".cast("double")), + avg($"b".cast("double")), + count_distinct($"a"), + count_distinct($"b")), + Row(12.0, 1.5, 3, 2)) + } + + // This test is applicable to velox backend. For CH backend, the replacement is disabled. + test( + GlutenTestConstants.GLUTEN_TEST + + "use gluten hash agg to replace vanilla spark sort agg") { + + withSQLConf(("spark.gluten.sql.columnar.force.hashagg", "false")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + // SortAggregateExec is expected to be used for string type input. + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + assert(find(df.queryExecution.executedPlan)(_.isInstanceOf[SortAggregateExec]).isDefined) + } + + withSQLConf(("spark.gluten.sql.columnar.force.hashagg", "true")) { + Seq("A", "B", "C", "D").toDF("col1").createOrReplaceTempView("t1") + val df = spark.sql("select max(col1) from t1") + checkAnswer(df, Row("D") :: Nil) + // Sort agg is expected to be replaced by gluten's hash agg. + assert( + find(df.queryExecution.executedPlan)( + _.isInstanceOf[HashAggregateExecBaseTransformer]).isDefined) + } + } + + test("mixed supported and unsupported aggregate functions") { + withUserDefinedFunction(("udaf_sum", true)) { + spark.udf.register( + "udaf_sum", + udaf(new Aggregator[JLong, JLong, JLong] { + override def zero: JLong = 0 + override def reduce(b: JLong, a: JLong): JLong = a + b + override def merge(b1: JLong, b2: JLong): JLong = b1 + b2 + override def finish(reduction: JLong): JLong = reduction + override def bufferEncoder: Encoder[JLong] = Encoders.LONG + override def outputEncoder: Encoder[JLong] = Encoders.LONG + }) + ) + + val df = spark.sql("SELECT a, udaf_sum(b), max(b) FROM testData2 group by a") + checkAnswer(df, Row(1, 3, 2) :: Row(2, 3, 2) :: Row(3, 3, 2) :: Nil) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala new file mode 100644 index 000000000000..9367fab17f2b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameAsOfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameAsOfJoinSuite extends DataFrameAsOfJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala new file mode 100644 index 000000000000..7464968cba51 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameComplexTypeSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameComplexTypeSuite extends DataFrameComplexTypeSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala new file mode 100644 index 000000000000..44981e1cee71 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameFunctionsSuite extends DataFrameFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala new file mode 100644 index 000000000000..663a6111b0d0 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameHintSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameHintSuite extends DataFrameHintSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala new file mode 100644 index 000000000000..2a6e367bc08a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameImplicitsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameImplicitsSuite extends DataFrameImplicitsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.scala new file mode 100644 index 000000000000..6581d7f2d88d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameJoinSuite.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 + +class GlutenDataFrameJoinSuite extends DataFrameJoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Supports multi-part names for broadcast hint resolution" + ) +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala new file mode 100644 index 000000000000..424087c8de89 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameNaFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameNaFunctionsSuite extends DataFrameNaFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.scala new file mode 100644 index 000000000000..b484e0dce045 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFramePivotSuite.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.spark.sql + +import org.apache.spark.sql.functions._ + +class GlutenDataFramePivotSuite extends DataFramePivotSuite with GlutenSQLTestsTrait { + + // This test is ported from vanilla spark with pos value (1-based) changed from 0 to 1 for + // substring. In vanilla spark, pos=0 has same effectiveness as pos=1. But in velox, pos=0 + // will return an empty string as substring result. + test("pivot with column definition in groupby - using pos=1") { + val df = courseSales + .groupBy(substring(col("course"), 1, 1).as("foo")) + .pivot("year", Seq(2012, 2013)) + .sum("earnings") + .queryExecution + .executedPlan + + checkAnswer( + courseSales + .groupBy(substring(col("course"), 1, 1).as("foo")) + .pivot("year", Seq(2012, 2013)) + .sum("earnings"), + Row("d", 15000.0, 48000.0) :: Row("J", 20000.0, 30000.0) :: Nil + ) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala new file mode 100644 index 000000000000..d0d856b5167b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameRangeSuite.scala @@ -0,0 +1,26 @@ +/* + * 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 + +class GlutenDataFrameRangeSuite extends DataFrameRangeSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + "Cancelling stage in a query with Range", + "SPARK-20430 Initialize Range parameters in a driver side" + ) + +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala new file mode 100644 index 000000000000..61cc4bc4c080 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSelfJoinSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameSelfJoinSuite extends DataFrameSelfJoinSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala new file mode 100644 index 000000000000..d76d8b21cdcf --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSessionWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenDataFrameSessionWindowingSuite + extends DataFrameSessionWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala new file mode 100644 index 000000000000..d51d1034b01c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSetOperationsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenDataFrameSetOperationsSuite + extends DataFrameSetOperationsSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala new file mode 100644 index 000000000000..bab8e9b83cb2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameStatSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameStatSuite extends DataFrameStatSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala new file mode 100644 index 000000000000..fde79e3d0f18 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameSuite.scala @@ -0,0 +1,400 @@ +/* + * 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 io.glutenproject.execution.{ProjectExecTransformer, WholeStageTransformer} + +import org.apache.spark.SparkException +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.expressions.{EqualTo, Expression} +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData2 +import org.apache.spark.sql.types.StringType + +import scala.util.Random + +class GlutenDataFrameSuite extends DataFrameSuite with GlutenSQLTestsTrait { + + test(GlutenTestConstants.GLUTEN_TEST + "repartitionByRange") { + val partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + import testImplicits._ + val data1d = Random.shuffle(0.to(partitionNum - 1)) + val data2d = data1d.map(i => (i, data1d.size - i)) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".asc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, $"val".desc) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, data1d.size - 1 - i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(42)) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(0, i))) + + checkAnswer( + data1d + .toDF("val") + .repartitionByRange(data1d.size, lit(null), $"val".asc, rand()) + .select(spark_partition_id().as("id"), $"val"), + data1d.map(i => Row(i, i))) + + // .repartitionByRange() assumes .asc by default if no explicit sort order is specified + checkAnswer( + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b") + .select(spark_partition_id().as("id"), $"a", $"b"), + data2d + .toDF("a", "b") + .repartitionByRange(data2d.size, $"a".desc, $"b".asc) + .select(spark_partition_id().as("id"), $"a", $"b") + ) + + // at least one partition-by expression must be specified + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size) + } + intercept[IllegalArgumentException] { + data1d.toDF("val").repartitionByRange(data1d.size, Seq.empty: _*) + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "distributeBy and localSort") { + import testImplicits._ + val data = spark.sparkContext.parallelize((1 to 100).map(i => TestData2(i % 10, i))).toDF() + + /** partitionNum = 1 */ + var partitionNum = 1 + val original = testData.repartition(partitionNum) + assert(original.rdd.partitions.length == partitionNum) + + // Distribute into one partition and order by. This partition should contain all the values. + val df6 = data.repartition(partitionNum, $"a").sortWithinPartitions("b") + // Walk each partition and verify that it is sorted ascending and not globally sorted. + df6.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (!allSequential) { + throw new SparkException("Partition should contain all sequential values") + } + } + + /** partitionNum = 5 */ + partitionNum = 5 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df = original.repartition(partitionNum, $"key") + assert(df.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df.select()) + + // Distribute and order by. + val df4 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".desc) + // Walk each partition and verify that it is sorted descending and does not contain all + // the values. + df4.rdd.foreachPartition { + p => + // Skip empty partition + if (p.hasNext) { + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue < v) throw new SparkException("Partition is not ordered.") + if (v + 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be globally ordered") + } + } + } + + /** partitionNum = 10 */ + partitionNum = 10 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + val df2 = original.repartition(partitionNum, $"key") + assert(df2.rdd.partitions.length == partitionNum) + checkAnswer(original.select(), df2.select()) + } + + // Group by the column we are distributed by. This should generate a plan with no exchange + // between the aggregates + val df3 = testData.repartition($"key").groupBy("key").count() + verifyNonExchangingAgg(df3) + verifyNonExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key", "value") + .count()) + + // Grouping by just the first distributeBy expr, need to exchange. + verifyExchangingAgg( + testData + .repartition($"key", $"value") + .groupBy("key") + .count()) + + /** partitionNum = 2 */ + partitionNum = 2 + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> partitionNum.toString) { + // Distribute and order by with multiple order bys + val df5 = data.repartition(partitionNum, $"a").sortWithinPartitions($"b".asc, $"a".asc) + // Walk each partition and verify that it is sorted ascending + df5.rdd.foreachPartition { + p => + var previousValue: Int = -1 + var allSequential: Boolean = true + p.foreach { + r => + val v: Int = r.getInt(1) + if (previousValue != -1) { + if (previousValue > v) throw new SparkException("Partition is not ordered.") + if (v - 1 != previousValue) allSequential = false + } + previousValue = v + } + if (allSequential) throw new SparkException("Partition should not be all sequential") + } + } + } + + test(GLUTEN_TEST + "reuse exchange") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { + val df = spark.range(100).toDF() + val join = df.join(df, "id") + val plan = join.queryExecution.executedPlan + checkAnswer(join, df) + assert(collect(join.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size === 1) + assert(collect(join.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size === 1) + val broadcasted = broadcast(join) + val join2 = join.join(broadcasted, "id").join(broadcasted, "id") + checkAnswer(join2, df) + assert(collect(join2.queryExecution.executedPlan) { + // replace ShuffleExchangeExec + case e: ColumnarShuffleExchangeExec => true + }.size == 1) + assert(collect(join2.queryExecution.executedPlan) { + case e: ReusedExchangeExec => true + }.size == 4) + } + } + + /** Failed to check WholeStageCodegenExec, so we rewrite the UT. */ + test(GLUTEN_TEST + "SPARK-22520: support code generation for large CaseWhen") { + import org.apache.spark.sql.catalyst.dsl.expressions.StringToAttributeConversionHelper + val N = 30 + var expr1 = when(equalizer($"id", lit(0)), 0) + var expr2 = when(equalizer($"id", lit(0)), 10) + (1 to N).foreach { + i => + expr1 = expr1.when(equalizer($"id", lit(i)), -i) + expr2 = expr2.when(equalizer($"id", lit(i + 10)), i) + } + val df = spark.range(1).select(expr1, expr2.otherwise(0)) + checkAnswer(df, Row(0, 10) :: Nil) + // We check WholeStageTransformer instead of WholeStageCodegenExec + assert(df.queryExecution.executedPlan.find(_.isInstanceOf[WholeStageTransformer]).isDefined) + } + + import testImplicits._ + + private lazy val person2: DataFrame = Seq( + ("Bob", 16, 176), + ("Alice", 32, 164), + ("David", 60, 192), + ("Amy", 24, 180)).toDF("name", "age", "height") + + test(GLUTEN_TEST + "describe") { + val describeResult = Seq( + Row("count", "4", "4", "4"), + Row("mean", null, "33.0", "178.0"), + Row("stddev", null, "19.148542155126762", "11.547005383792516"), + Row("min", "Alice", "16", "164"), + Row("max", "David", "60", "192") + ) + + val emptyDescribeResult = Seq( + Row("count", "0", "0", "0"), + Row("mean", null, null, null), + Row("stddev", null, null, null), + Row("min", null, null, null), + Row("max", null, null, null)) + + val aggResult = Seq( + Row("4", "33.0", "19.148542155126762", "16", "60") + ) + + def getSchemaAsSeq(df: DataFrame): Seq[String] = df.schema.map(_.name) + + Seq("true", "false").foreach { + ansiEnabled => + withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled) { + val describeAllCols = person2.describe() + assert(getSchemaAsSeq(describeAllCols) === Seq("summary", "name", "age", "height")) + checkAnswer(describeAllCols, describeResult) + // All aggregate value should have been cast to string + describeAllCols.collect().foreach { + row => + row.toSeq.foreach { + value => + if (value != null) { + assert( + value.isInstanceOf[String], + "expected string but found " + value.getClass) + } + } + } + + val describeOneCol = person2.describe("age") + assert(getSchemaAsSeq(describeOneCol) === Seq("summary", "age")) + val aggOneCol = person2.agg( + count("age").cast(StringType), + avg("age").cast(StringType), + stddev_samp("age").cast(StringType), + min("age").cast(StringType), + max("age").cast(StringType)) + checkAnswer(aggOneCol, aggResult) + + val describeNoCol = person2.select().describe() + assert(getSchemaAsSeq(describeNoCol) === Seq("summary")) + checkAnswer(describeNoCol, describeResult.map { case Row(s, _, _, _) => Row(s) }) + + val emptyDescription = person2.limit(0).describe() + assert(getSchemaAsSeq(emptyDescription) === Seq("summary", "name", "age", "height")) + checkAnswer(emptyDescription, emptyDescribeResult) + } + } + } + + test( + GLUTEN_TEST + + "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) + } + + // 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())).seq + df = spark.sql("select cast(col1 as binary) from t1") + checkResult(df, expectedBinaryResult) + } + + private def withExpr(newExpr: Expression): Column = new Column(newExpr) + + def equalizer(expr: Expression, other: Any): Column = withExpr { + val right = lit(other).expr + if (expr == right) { + logWarning( + s"Constructing trivially true equals predicate, '$expr = $right'. " + + "Perhaps you need to use aliases.") + } + EqualTo(expr, right) + } + + private def verifyNonExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case agg: HashAggregateExec => + atFirstAgg = !atFirstAgg + case _ => + if (atFirstAgg) { + fail("Should not have operators between the two aggregations") + } + } + } + + private def verifyExchangingAgg(df: DataFrame): Unit = { + var atFirstAgg: Boolean = false + df.queryExecution.executedPlan.foreach { + case _: HashAggregateExec => + if (atFirstAgg) { + fail("Should not have back to back Aggregates") + } + atFirstAgg = true + case _: ShuffleExchangeExec => atFirstAgg = false + case _ => + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala new file mode 100644 index 000000000000..f2833a357cd2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTimeWindowingSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenDataFrameTimeWindowingSuite + extends DataFrameTimeWindowingSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala new file mode 100644 index 000000000000..d850a3c64d69 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameTungstenSuite.scala @@ -0,0 +1,34 @@ +/* + * 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.types._ + +class GlutenDataFrameTungstenSuite extends DataFrameTungstenSuite with GlutenSQLTestsTrait { + + test("Map type with struct type as key") { + val kv = Map(Row(1, 2L) -> Seq("v")) + val data = sparkContext.parallelize(Seq(Row(1, kv))) + val schema = new StructType() + .add("a", IntegerType) + .add( + "b", + MapType(new StructType().add("k1", IntegerType).add("k2", LongType), ArrayType(StringType))) + val df = spark.createDataFrame(data, schema) + assert(df.select("b").first() === Row(kv)) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala new file mode 100644 index 000000000000..3ba990d2eea6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFramesSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenDataFrameWindowFramesSuite + extends DataFrameWindowFramesSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala new file mode 100644 index 000000000000..18d6fce39969 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWindowFunctionsSuite.scala @@ -0,0 +1,157 @@ +/* + * 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.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.expressions.Window +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +class GlutenDataFrameWindowFunctionsSuite + extends DataFrameWindowFunctionsSuite + with GlutenSQLTestsTrait { + + import testImplicits._ + + test( + GLUTEN_TEST + + "covar_samp, var_samp (variance), stddev_samp (stddev) functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + covar_samp("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + variance("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_samp("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev("value1").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + Seq( + Row("a", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("b", -50.0, 50.0, 50.0, 7.0710678118654755, 7.0710678118654755), + Row("c", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("f", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("g", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544), + Row("h", 24.0, 12.0, 12.0, 3.4641016151377544, 3.4641016151377544) + ) + ) + } + } + + test(GLUTEN_TEST + "corr, covar_pop, stddev_pop functions in specific window") { + withSQLConf(SQLConf.LEGACY_STATISTICAL_AGGREGATE.key -> "true") { + val df = Seq( + ("a", "p1", 10.0, 20.0), + ("b", "p1", 20.0, 10.0), + ("c", "p2", 20.0, 20.0), + ("d", "p2", 20.0, 20.0), + ("e", "p3", 0.0, 0.0), + ("f", "p3", 6.0, 12.0), + ("g", "p3", 6.0, 12.0), + ("h", "p3", 8.0, 16.0) + ).toDF("key", "partitionId", "value1", "value2") + checkAnswer( + df.select( + $"key", + corr("value1", "value2").over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + covar_pop("value1", "value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value1") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + var_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)), + stddev_pop("value2") + .over( + Window + .partitionBy("partitionId") + .orderBy("key") + .rowsBetween(Window.unboundedPreceding, Window.unboundedFollowing)) + ), + + // As stddev_pop(expr) = sqrt(var_pop(expr)) + // the "stddev_pop" column can be calculated from the "var_pop" column. + // + // As corr(expr1, expr2) = covar_pop(expr1, expr2) / (stddev_pop(expr1) * stddev_pop(expr2)) + // the "corr" column can be calculated from the "covar_pop" and the two "stddev_pop" columns + Seq( + Row("a", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("b", -1.0, -25.0, 25.0, 5.0, 25.0, 5.0), + Row("c", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("d", null, 0.0, 0.0, 0.0, 0.0, 0.0), + Row("e", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("f", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("g", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0), + Row("h", 1.0, 18.0, 9.0, 3.0, 36.0, 6.0) + ) + ) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala new file mode 100644 index 000000000000..ddae3139d06b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDataFrameWriterV2Suite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDataFrameWriterV2Suite extends DataFrameWriterV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala new file mode 100644 index 000000000000..8a9a6b5756e9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetAggregatorSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDatasetAggregatorSuite extends DatasetAggregatorSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala new file mode 100644 index 000000000000..848560192722 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDatasetCacheSuite extends DatasetCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala new file mode 100644 index 000000000000..a9d1bd29cead --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetOptimizationSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDatasetOptimizationSuite extends DatasetOptimizationSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala new file mode 100644 index 000000000000..c7463dcef75f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetPrimitiveSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDatasetPrimitiveSuite extends DatasetPrimitiveSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala new file mode 100644 index 000000000000..6749227ed79d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSerializerRegistratorSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenDatasetSerializerRegistratorSuite + extends DatasetSerializerRegistratorSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala new file mode 100644 index 000000000000..49310432d136 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDatasetSuite.scala @@ -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.spark.sql + +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec + +class GlutenDatasetSuite extends DatasetSuite with GlutenSQLTestsTrait { + import testImplicits._ + + test("Gluten: dropDuplicates: columns with same column name") { + val ds1 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + val ds2 = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS() + // The dataset joined has two columns of the same name "_2". + val joined = ds1.join(ds2, "_1").select(ds1("_2").as[Int], ds2("_2").as[Int]) + // Using the checkDatasetUnorderly method to sort the result in Gluten. + checkDatasetUnorderly(joined.dropDuplicates(), (1, 2), (1, 1), (2, 1), (2, 2)) + } + + test("Gluten: groupBy.as") { + val df1 = Seq(DoubleData(1, "one"), DoubleData(2, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + val df2 = Seq(DoubleData(5, "one"), DoubleData(1, "two"), DoubleData(3, "three")) + .toDS() + .repartition($"id") + .sortWithinPartitions("id") + + val df3 = df1 + .groupBy("id") + .as[Int, DoubleData] + .cogroup(df2.groupBy("id").as[Int, DoubleData]) { + case (key, data1, data2) => + if (key == 1) { + Iterator(DoubleData(key, (data1 ++ data2).foldLeft("")((cur, next) => cur + next.val1))) + } else Iterator.empty + } + checkDataset(df3, DoubleData(1, "onetwo")) + + // Assert that no extra shuffle introduced by cogroup. + val exchanges = collect(df3.queryExecution.executedPlan) { + case h: ColumnarShuffleExchangeExec => h + } + // Assert the number of ColumnarShuffleExchangeExec + // instead of ShuffleExchangeExec in Gluten. + assert(exchanges.size == 2) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala new file mode 100644 index 000000000000..3e1e9a19f14d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDateFunctionsSuite.scala @@ -0,0 +1,208 @@ +/* + * 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.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +import java.sql.{Date, Timestamp} +import java.time.{LocalDateTime, ZoneId} +import java.util.concurrent.TimeUnit + +class GlutenDateFunctionsSuite extends DateFunctionsSuite with GlutenSQLTestsTrait { + import testImplicits._ + + private def secs(millis: Long): Long = TimeUnit.MILLISECONDS.toSeconds(millis) + + test(GLUTEN_TEST + "unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val ntzTs1 = LocalDateTime.parse("2015-07-24T10:00:00.3") + val ntzTs2 = LocalDateTime.parse("2015-07-25T02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, ntzTs1, s1, ss1), (date2, ts2, ntzTs2, s2, ss2)).toDF( + "d", + "ts", + "ntzTs", + "s", + "ss") + checkAnswer( + df.select(unix_timestamp(col("ts"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ss"))), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("ntzTs"))), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.select(unix_timestamp(col("d"), fmt)), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.select(unix_timestamp(col("s"), fmt)), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("unix_timestamp(ntzTs)"), + Seq( + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs1)))), + Row(secs(DateTimeUtils.microsToMillis(DateTimeUtils.localDateTimeToMicros(ntzTs2)))) + ) + ) + checkAnswer( + df.selectExpr(s"unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.select(unix_timestamp(col("x"))), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr("unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), + Seq(Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // legacyParserPolicy is not respected by Gluten. + // invalid format + // val invalid = df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')") + // if (legacyParserPolicy == "legacy") { + // checkAnswer(invalid, + // Seq(Row(null), Row(null), Row(null), Row(null))) + // } else { + // val e = intercept[SparkUpgradeException](invalid.collect()) + // assert(e.getCause.isInstanceOf[IllegalArgumentException]) + // assert( e.getMessage.contains( + // "You may get a different result due to the upgrading to Spark")) + // } + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + val now = sql("select unix_timestamp()").collect().head.getLong(0) + checkAnswer( + sql(s"select timestamp_seconds($now)"), + Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) + } + } + } + + test(GLUTEN_TEST + "to_unix_timestamp") { + Seq("corrected", "legacy").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> ZoneId.systemDefault().toString + ) { + val date1 = Date.valueOf("2015-07-24") + val date2 = Date.valueOf("2015-07-25") + val ts1 = Timestamp.valueOf("2015-07-24 10:00:00.3") + val ts2 = Timestamp.valueOf("2015-07-25 02:02:02.2") + val s1 = "2015/07/24 10:00:00.5" + val s2 = "2015/07/25 02:02:02.6" + val ss1 = "2015-07-24 10:00:00" + val ss2 = "2015-07-25 02:02:02" + val fmt = "yyyy/MM/dd HH:mm:ss.S" + val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") + checkAnswer( + df.selectExpr("to_unix_timestamp(ts)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr("to_unix_timestamp(ss)"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), + Seq(Row(secs(date1.getTime)), Row(secs(date2.getTime)))) + checkAnswer( + df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), + Seq(Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) + + val x1 = "2015-07-24 10:00:00" + val x2 = "2015-25-07 02:02:02" + val x3 = "2015-07-24 25:02:02" + val x4 = "2015-24-07 26:02:02" + val ts3 = Timestamp.valueOf("2015-07-24 02:25:02") + val ts4 = Timestamp.valueOf("2015-07-24 00:10:00") + + val df1 = Seq(x1, x2, x3, x4).toDF("x") + checkAnswer( + df1.selectExpr("to_unix_timestamp(x)"), + Seq(Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) + checkAnswer( + df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), + Seq(Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) + + // February + val y1 = "2016-02-29" + val y2 = "2017-02-29" + val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") + val df2 = Seq(y1, y2).toDF("y") + checkAnswer( + df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), + Seq(Row(secs(ts5.getTime)), Row(null))) + + // Not consistent behavior with gluten + velox. + // invalid format + // val invalid = df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')") + // val e = intercept[IllegalArgumentException](invalid.collect()) + // assert(e.getMessage.contains('b')) + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala new file mode 100644 index 000000000000..b6428773f1d0 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDeprecatedAPISuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenDeprecatedAPISuite extends DeprecatedAPISuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala new file mode 100644 index 000000000000..9048c3845b35 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenDynamicPartitionPruningSuite.scala @@ -0,0 +1,744 @@ +/* + * 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 io.glutenproject.GlutenConfig +import io.glutenproject.execution.{BatchScanExecTransformer, FileSourceScanExecTransformer, FilterExecTransformerBase} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.catalyst.expressions.{DynamicPruningExpression, Expression} +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode.{CODEGEN_ONLY, NO_CODEGEN} +import org.apache.spark.sql.catalyst.plans.ExistenceJoin +import org.apache.spark.sql.connector.catalog.InMemoryTableCatalog +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, ReusedExchangeExec} +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +abstract class GlutenDynamicPartitionPruningSuiteBase + extends DynamicPartitionPruningSuiteBase + with GlutenSQLTestsTrait { + + import testImplicits._ + + override def beforeAll(): Unit = { + prepareWorkDir() + super.beforeAll() + spark.sparkContext.setLogLevel("WARN") + } + + override def testNameBlackList: Seq[String] = Seq( + // overwritten with different plan + "Make sure dynamic pruning works on uncorrelated queries", + "Subquery reuse across the whole plan", + // struct join key not supported, fell-back to Vanilla join + "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type" + ) + + // === Following cases override super class's cases === + + ignore(GLUTEN_TEST + "DPP should not be rewritten as an existential join") { + // ignored: BroadcastHashJoinExec is from Vanilla Spark + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_FALLBACK_FILTER_RATIO.key -> "1.5", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(s""" + |SELECT * FROM product p WHERE p.store_id NOT IN + | (SELECT f.store_id FROM fact_sk f JOIN dim_store d ON + | f.store_id = d.store_id + | WHERE d.state_province = 'NL' + | ) + """.stripMargin) + + val found = df.queryExecution.executedPlan.find { + case _ @BroadcastHashJoinExec(_, _, _: ExistenceJoin, _, _, _, _, _) => true + case _ => false + } + + assert(found.isEmpty) + } + } + + test(GLUTEN_TEST + "no partition pruning when the build side is a stream") { + withTable("fact") { + val input = MemoryStream[Int] + val stream = input.toDF.select($"value".as("one"), ($"value" * 3).as("code")) + spark + .range(100) + .select($"id", ($"id" + 1).as("one"), ($"id" + 2).as("two"), ($"id" + 3).as("three")) + .write + .partitionBy("one") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + val table = sql("SELECT * from fact f") + + // join a partitioned table with a stream + val joined = table.join(stream, Seq("one")).where("code > 40") + val query = joined.writeStream.format("memory").queryName("test").start() + input.addData(1, 10, 20, 40, 50) + try { + query.processAllAvailable() + } finally { + query.stop() + } + // search dynamic pruning predicates on the executed plan + val plan = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.lastExecution.executedPlan + val ret = plan.find { + case s: FileSourceScanExecTransformer => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case s: FileSourceScanExec => + s.partitionFilters.exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + } + assert(ret.isDefined == false) + } + } + + test(GLUTEN_TEST + "Make sure dynamic pruning works on uncorrelated queries") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + val df = sql(""" + |SELECT d.store_id, + | SUM(f.units_sold), + | (SELECT SUM(f.units_sold) + | FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + | WHERE d.country = 'US') AS total_prod + |FROM fact_stats f JOIN dim_stats d ON d.store_id = f.store_id + |WHERE d.country = 'US' + |GROUP BY 1 + """.stripMargin) + checkAnswer(df, Row(4, 50, 70) :: Row(5, 10, 70) :: Row(6, 10, 70) :: Nil) + + val plan = df.queryExecution.executedPlan + val countSubqueryBroadcasts = + collectWithSubqueries(plan) { + case _: SubqueryBroadcastExec => 1 + case _: ColumnarSubqueryBroadcastExec => 1 + }.sum + + val countReusedSubqueryBroadcasts = + collectWithSubqueries(plan) { + case ReusedSubqueryExec(_: SubqueryBroadcastExec) => 1 + case ReusedSubqueryExec(_: ColumnarSubqueryBroadcastExec) => 1 + }.sum + + assert(countSubqueryBroadcasts == 1) + assert(countReusedSubqueryBroadcasts == 1) + } + } + + test( + GLUTEN_TEST + "SPARK-32509: Unused Dynamic Pruning filter shouldn't affect " + + "canonicalization and exchange reuse") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "true") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val df = sql(""" WITH view1 as ( + | SELECT f.store_id FROM fact_stats f WHERE f.units_sold = 70 + | ) + | + | SELECT * FROM view1 v1 join view1 v2 WHERE v1.store_id = v2.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + val reuseExchangeNodes = collect(df.queryExecution.executedPlan) { + case se: ReusedExchangeExec => se + } + assert( + reuseExchangeNodes.size == 1, + "Expected plan to contain 1 ReusedExchangeExec " + + s"nodes. Found ${reuseExchangeNodes.size}") + + checkAnswer(df, Row(15, 15) :: Nil) + } + } + } + + test(GLUTEN_TEST + "SPARK-32659: Fix the data issue when pruning DPP on non-atomic type") { + Seq(NO_CODEGEN, CODEGEN_ONLY).foreach { + mode => + Seq(true, false).foreach { + pruning => + withSQLConf( + SQLConf.CODEGEN_FACTORY_MODE.key -> mode.toString, + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> s"$pruning") { + Seq("struct", "array").foreach { + dataType => + val df = sql( + s""" + |SELECT f.date_id, f.product_id, f.units_sold, f.store_id FROM fact_stats f + |JOIN dim_stats s + |ON $dataType(f.store_id) = $dataType(s.store_id) WHERE s.country = 'DE' + """.stripMargin) + + if (pruning) { + df.collect() + + val plan = df.queryExecution.executedPlan + val dpExprs = collectDynamicPruningExpressions(plan) + val hasSubquery = dpExprs.exists { + case InSubqueryExec(_, _: SubqueryExec, _, _, _, _) => true + case _ => false + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (false) "Should" else "Shouldn't" + assert( + !hasSubquery, + s"$hasFilter trigger DPP with a subquery duplicate:\n${df.queryExecution}") + val hasBroadcast = if (true) "Should" else "Shouldn't" + assert( + subqueryBroadcast.nonEmpty, + s"$hasBroadcast trigger DPP " + + s"with a reused broadcast exchange:\n${df.queryExecution}") + + subqueryBroadcast.foreach { + s => + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case BroadcastQueryStageExec( + _, + _: ReusedExchangeExec, + _ + ) => // reuse check ok. + case b: BroadcastExchangeLike => + val hasReuse = plan.find { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + }.isDefined + // assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case a: AdaptiveSparkPlanExec => + val broadcastQueryStage = collectFirst(a) { + case b: BroadcastQueryStageExec => b + } + val broadcastPlan = broadcastQueryStage.get.broadcast + val hasReuse = find(plan) { + case ReusedExchangeExec(_, e) => e eq broadcastPlan + case b: BroadcastExchangeLike => b eq broadcastPlan + case _ => false + }.isDefined + // assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case _ => + fail(s"Invalid child node found in\n$s") + } + } + + val isMainQueryAdaptive = plan.isInstanceOf[AdaptiveSparkPlanExec] + subqueriesAll(plan).filterNot(subqueryBroadcast.contains).foreach { + s => + val subquery = s match { + case r: ReusedSubqueryExec => r.child + case o => o + } + assert( + subquery + .find(_.isInstanceOf[AdaptiveSparkPlanExec]) + .isDefined == isMainQueryAdaptive) + } + } else { + checkPartitionPruningPredicate(df, false, false) + } + + checkAnswer( + df, + Row(1030, 2, 10, 3) :: + Row(1040, 2, 50, 3) :: + Row(1050, 2, 50, 3) :: + Row(1060, 2, 50, 3) :: Nil) + } + } + } + } + } + + // === Following methods override super class's methods === + + override protected def collectDynamicPruningExpressions(plan: SparkPlan): Seq[Expression] = { + flatMap(plan) { + case s: FileSourceScanExecTransformer => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: FileSourceScanExec => + s.partitionFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExecTransformer => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case s: BatchScanExec => + s.runtimeFilters.collect { case d: DynamicPruningExpression => d.child } + case _ => Nil + } + } + + override def checkPartitionPruningPredicate( + df: DataFrame, + withSubquery: Boolean, + withBroadcast: Boolean): Unit = { + df.collect() + + val plan = df.queryExecution.executedPlan + val dpExprs = collectDynamicPruningExpressions(plan) + val hasSubquery = dpExprs.exists { + case InSubqueryExec(_, _: SubqueryExec, _, _, _, _) => true + case _ => false + } + val subqueryBroadcast = dpExprs.collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => b + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => b + } + + val hasFilter = if (withSubquery) "Should" else "Shouldn't" + assert( + hasSubquery == withSubquery, + s"$hasFilter trigger DPP with a subquery duplicate:\n${df.queryExecution}") + val hasBroadcast = if (withBroadcast) "Should" else "Shouldn't" + assert( + subqueryBroadcast.nonEmpty == withBroadcast, + s"$hasBroadcast trigger DPP with a reused broadcast exchange:\n${df.queryExecution}") + + subqueryBroadcast.foreach { + s => + s.child match { + case _: ReusedExchangeExec => // reuse check ok. + case BroadcastQueryStageExec(_, _: ReusedExchangeExec, _) => // reuse check ok. + case b: BroadcastExchangeLike => + val hasReuse = plan.find { + case ReusedExchangeExec(_, e) => e eq b + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case a: AdaptiveSparkPlanExec => + val broadcastQueryStage = collectFirst(a) { case b: BroadcastQueryStageExec => b } + val broadcastPlan = broadcastQueryStage.get.broadcast + val hasReuse = find(plan) { + case ReusedExchangeExec(_, e) => e eq broadcastPlan + case b: BroadcastExchangeLike => b eq broadcastPlan + case _ => false + }.isDefined + assert(hasReuse, s"$s\nshould have been reused in\n$plan") + case _ => + fail(s"Invalid child node found in\n$s") + } + } + + val isMainQueryAdaptive = plan.isInstanceOf[AdaptiveSparkPlanExec] + subqueriesAll(plan).filterNot(subqueryBroadcast.contains).foreach { + s => + val subquery = s match { + case r: ReusedSubqueryExec => r.child + case o => o + } + assert( + subquery.find(_.isInstanceOf[AdaptiveSparkPlanExec]).isDefined == isMainQueryAdaptive) + } + } + + override def checkDistinctSubqueries(df: DataFrame, n: Int): Unit = { + df.collect() + + val buf = collectDynamicPruningExpressions(df.queryExecution.executedPlan).collect { + case InSubqueryExec(_, b: SubqueryBroadcastExec, _, _, _, _) => + b.index + case InSubqueryExec(_, b: ColumnarSubqueryBroadcastExec, _, _, _, _) => + b.index + } + assert(buf.distinct.size == n) + } + + override def checkUnpushedFilters(df: DataFrame): Boolean = { + find(df.queryExecution.executedPlan) { + case FilterExec(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case transformer: FilterExecTransformerBase => + splitConjunctivePredicates(transformer.cond).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case FilterTransformer(condition, _) => + splitConjunctivePredicates(condition).exists { + case _: DynamicPruningExpression => true + case _ => false + } + case _ => false + }.isDefined + } + + object FilterTransformer { + def unapply(plan: SparkPlan): Option[(Expression, SparkPlan)] = { + plan match { + case transformer: FilterExecTransformerBase => + Some((transformer.cond, transformer.input)) + case _ => None + } + } + } +} + +abstract class GlutenDynamicPartitionPruningV1Suite extends GlutenDynamicPartitionPruningSuiteBase { + + import testImplicits._ + + /** Check the static scan metrics with and without DPP */ + test("static scan metrics", DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + val allFilesNum = scan1.metrics("numFiles").value + val allFilesSize = scan1.metrics("filesSize").value + assert(scan1.metrics("numPartitions").value === numPartitions) + assert(scan1.metrics("pruningTime").value === -1) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum = scan2.metrics("numFiles").value + val partFilesSize = scan2.metrics("filesSize").value + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(scan2.metrics("numPartitions").value === 1) + assert(scan2.metrics("pruningTime").value === -1) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + assert(scan3.metrics("staticFilesNum").value == allFilesNum) + assert(scan3.metrics("staticFilesSize").value == allFilesSize) + assert(scan3.metrics("numFiles").value == partFilesNum) + assert(scan3.metrics("filesSize").value == partFilesSize) + assert(scan3.metrics("numPartitions").value === 1) + assert(scan3.metrics("pruningTime").value !== -1) + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOff + extends GlutenDynamicPartitionPruningV1Suite + with DisableAdaptiveExecutionSuite { + + import testImplicits._ + + test( + GLUTEN_TEST + "static scan metrics", + DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + // "spark.gluten.enabled" -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("fact", "dim") { + val numPartitions = 10 + + spark + .range(10) + .map(x => Tuple3(x, x + 1, 0)) + .toDF("did", "d1", "d2") + .write + .format(tableFormat) + .mode("overwrite") + .saveAsTable("dim") + + spark + .range(100) + .map(x => Tuple2(x, x % numPartitions)) + .toDF("f1", "fid") + .write + .partitionBy("fid") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("fact") + + def getFactScan(plan: SparkPlan): SparkPlan = { + val scanOption = + find(plan) { + case s: FileSourceScanExecTransformer => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: FileSourceScanExec => + s.output.exists(_.find(_.argString(maxFields = 100).contains("fid")).isDefined) + case s: BatchScanExecTransformer => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case s: BatchScanExec => + // we use f1 col for v2 tables due to schema pruning + s.output.exists(_.find(_.argString(maxFields = 100).contains("f1")).isDefined) + case _ => false + } + assert(scanOption.isDefined) + scanOption.get + } + + // No dynamic partition pruning, so no static metrics + // All files in fact table are scanned + val df1 = sql("SELECT sum(f1) FROM fact") + df1.collect() + val scan1 = getFactScan(df1.queryExecution.executedPlan) + assert(!scan1.metrics.contains("staticFilesNum")) + assert(!scan1.metrics.contains("staticFilesSize")) + val allFilesNum = scan1.metrics("numFiles").value + val allFilesSize = scan1.metrics("filesSize").value + assert(scan1.metrics("numPartitions").value === numPartitions) + assert(scan1.metrics("pruningTime").value === -1) + + // No dynamic partition pruning, so no static metrics + // Only files from fid = 5 partition are scanned + val df2 = sql("SELECT sum(f1) FROM fact WHERE fid = 5") + df2.collect() + val scan2 = getFactScan(df2.queryExecution.executedPlan) + assert(!scan2.metrics.contains("staticFilesNum")) + assert(!scan2.metrics.contains("staticFilesSize")) + val partFilesNum = scan2.metrics("numFiles").value + val partFilesSize = scan2.metrics("filesSize").value + assert(0 < partFilesNum && partFilesNum < allFilesNum) + assert(0 < partFilesSize && partFilesSize < allFilesSize) + assert(scan2.metrics("numPartitions").value === 1) + assert(scan2.metrics("pruningTime").value === -1) + + // Dynamic partition pruning is used + // Static metrics are as-if reading the whole fact table + // "Regular" metrics are as-if reading only the "fid = 5" partition + val df3 = sql("SELECT sum(f1) FROM fact, dim WHERE fid = did AND d1 = 6") + df3.collect() + val scan3 = getFactScan(df3.queryExecution.executedPlan) + assert(scan3.metrics("staticFilesNum").value == allFilesNum) + assert(scan3.metrics("staticFilesSize").value == allFilesSize) + assert(scan3.metrics("numFiles").value == partFilesNum) + assert(scan3.metrics("filesSize").value == partFilesSize) + assert(scan3.metrics("numPartitions").value === 1) + assert(scan3.metrics("pruningTime").value !== -1) + } + } + } + + test( + GLUTEN_TEST + "Subquery reuse across the whole plan", + DisableAdaptiveExecution("DPP in AQE must reuse broadcast")) { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + withTable("df1", "df2") { + spark + .range(100) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df1") + + spark + .range(10) + .select(col("id"), col("id").as("k")) + .write + .partitionBy("k") + .format(tableFormat) + .mode("overwrite") + .saveAsTable("df2") + + val df = sql(""" + |SELECT df1.id, df2.k + |FROM df1 JOIN df2 ON df1.k = df2.k + |WHERE df2.id < (SELECT max(id) FROM df2 WHERE id <= 2) + |""".stripMargin) + + checkPartitionPruningPredicate(df, true, false) + + checkAnswer(df, Row(0, 0) :: Row(1, 1) :: Nil) + + val plan = df.queryExecution.executedPlan + + val subqueryIds = plan.collectWithSubqueries { case s: SubqueryExec => s.id } + val reusedSubqueryIds = plan.collectWithSubqueries { + case rs: ReusedSubqueryExec => rs.child.id + } + + // By default Gluten pushes more filters than vanilla Spark. + // + // See also io.glutenproject.execution.FilterHandler#applyFilterPushdownToScan + // See also DynamicPartitionPruningSuite.scala:1362 + assert(subqueryIds.size == 3, "Whole plan subquery reusing not working correctly") + assert(reusedSubqueryIds.size == 2, "Whole plan subquery reusing not working correctly") + assert( + reusedSubqueryIds.forall(subqueryIds.contains(_)), + "ReusedSubqueryExec should reuse an existing subquery") + } + } + } +} + +class GlutenDynamicPartitionPruningV1SuiteAEOn + extends GlutenDynamicPartitionPruningV1Suite + with EnableAdaptiveExecutionSuite { + + test("SPARK-39447: Avoid AssertionError in AdaptiveSparkPlanExec.doExecuteBroadcast") { + val df = sql(""" + |WITH empty_result AS ( + | SELECT * FROM fact_stats WHERE product_id < 0 + |) + |SELECT * + |FROM (SELECT /*+ SHUFFLE_MERGE(fact_sk) */ empty_result.store_id + | FROM fact_sk + | JOIN empty_result + | ON fact_sk.product_id = empty_result.product_id) t2 + | JOIN empty_result + | ON t2.store_id = empty_result.store_id + """.stripMargin) + + checkPartitionPruningPredicate(df, false, false) + checkAnswer(df, Nil) + } + + test( + "SPARK-37995: PlanAdaptiveDynamicPruningFilters should use prepareExecutedPlan " + + "rather than createSparkPlan to re-plan subquery") { + withSQLConf( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true", + SQLConf.DYNAMIC_PARTITION_PRUNING_REUSE_BROADCAST_ONLY.key -> "false", + SQLConf.EXCHANGE_REUSE_ENABLED.key -> "false" + ) { + val df = sql(""" + |SELECT f.date_id, f.store_id FROM fact_sk f + |JOIN dim_store s ON f.store_id = s.store_id AND s.country = 'NL' + |WHERE s.state_province != (SELECT max(state_province) FROM dim_stats) + """.stripMargin) + + checkPartitionPruningPredicate(df, true, false) + checkAnswer(df, Row(1000, 1) :: Row(1010, 2) :: Row(1020, 2) :: Nil) + } + } +} + +abstract class GlutenDynamicPartitionPruningV2Suite extends GlutenDynamicPartitionPruningSuiteBase { + override protected def runAnalyzeColumnCommands: Boolean = false + + override protected def initState(): Unit = { + spark.conf.set("spark.sql.catalog.testcat", classOf[InMemoryTableCatalog].getName) + spark.conf.set("spark.sql.defaultCatalog", "testcat") + } +} + +class GlutenDynamicPartitionPruningV2SuiteAEOff + extends GlutenDynamicPartitionPruningV2Suite + with DisableAdaptiveExecutionSuite + +class GlutenDynamicPartitionPruningV2SuiteAEOn + extends GlutenDynamicPartitionPruningV2Suite + with EnableAdaptiveExecutionSuite + +// Test DPP with file scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV1SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV1SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV1SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key, "false") + } +} + +// Test DPP with batch scan disabled by user for some reason, which can also mock the situation +// that scan is not transformable. +class GlutenDynamicPartitionPruningV2SuiteAEOnDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOn { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} + +// Same as above except AQE is off. +class GlutenDynamicPartitionPruningV2SuiteAEOffDisableScan + extends GlutenDynamicPartitionPruningV2SuiteAEOff { + override def sparkConf: SparkConf = { + super.sparkConf.set(GlutenConfig.COLUMNAR_BATCHSCAN_ENABLED.key, "false") + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala new file mode 100644 index 000000000000..0dd285c7426a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExpressionsSchemaSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenExpressionsSchemaSuite extends ExpressionsSchemaSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala new file mode 100644 index 000000000000..3c3b438f3cf0 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenExtraStrategiesSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenExtraStrategiesSuite extends ExtraStrategiesSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala new file mode 100644 index 000000000000..b4d693956c1f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileBasedDataSourceSuite.scala @@ -0,0 +1,177 @@ +/* + * 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.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf + +import org.apache.hadoop.fs.Path + +import scala.collection.mutable + +class GlutenFileBasedDataSourceSuite extends FileBasedDataSourceSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + // test data path is jar path, so failed, test code is same with spark + test("gluten Option recursiveFileLookup: disable partition inferring") { + val dataPath = getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/" + "test-data/text-partitioned" + + val df = spark.read + .format("binaryFile") + .option("recursiveFileLookup", true) + .load(dataPath) + + assert(!df.columns.contains("year"), "Expect partition inferring disabled") + val fileList = df.select("path").collect().map(_.getString(0)) + + val expectedFileList = Array( + dataPath + "/year=2014/data.txt", + dataPath + "/year=2015/data.txt" + ).map(path => "file:" + new Path(path).toString) + + assert(fileList.toSet === expectedFileList.toSet) + } + + test("gluten Spark native readers should respect spark.sql.caseSensitive - parquet") { + withTempDir { + dir => + val format = "parquet" + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" + withTable(tableName) { + val end = 5 + val data = spark.range(end).selectExpr("id as A", "id * 2 as b", "id * 3 as B") + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + data.write.format(format).mode("overwrite").save(tableDir) + } + sql(s"CREATE TABLE $tableName (a LONG, b LONG) USING $format LOCATION '$tableDir'") + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer(sql(s"select a from $tableName"), data.select("A")) + checkAnswer(sql(s"select A from $tableName"), data.select("A")) + + // TODO: gluten can catch exception in executor side, but cannot catch SparkException + // in Driver side + // RuntimeException is triggered at executor side, which is then wrapped as + // SparkException at driver side + // val e1 = intercept[SparkException] { + // sql(s"select b from $tableName").collect() + // } + // + // assert( + // e1.getCause.isInstanceOf[RuntimeException] && + // e1.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode """)) + // val e2 = intercept[SparkException] { + // sql(s"select B from $tableName").collect() + // } + // assert( + // e2.getCause.isInstanceOf[RuntimeException] && + // e2.getMessage.contains( + // """Found duplicate field(s) b in case-insensitive mode""")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer(sql(s"select a from $tableName"), (0 until end).map(_ => Row(null))) + checkAnswer(sql(s"select b from $tableName"), data.select("b")) + } + } + } + } + + test("gluten SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { + Seq(1.0, 0.5).foreach { + compressionFactor => + withSQLConf( + SQLConf.FILE_COMPRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "350") { + withTempPath { + workDir => + // the file size is 504 bytes + val workDirPath = workDir.getAbsolutePath + val data1 = Seq(100, 200, 300, 400).toDF("count") + data1.write.orc(workDirPath + "/data1") + val df1FromFile = spark.read.orc(workDirPath + "/data1") + val data2 = Seq(100, 200, 300, 400).toDF("count") + data2.write.orc(workDirPath + "/data2") + val df2FromFile = spark.read.orc(workDirPath + "/data2") + val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) + if (compressionFactor == 0.5) { + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.nonEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.isEmpty) + } else { + // compressionFactor is 1.0 + val bJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.isEmpty) + val smJoinExec = collect(joinedDF.queryExecution.executedPlan) { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } + } + } + } + } + + test("gluten SPARK-25237 compute correct input metrics in FileScanRDD") { + // TODO: Test CSV V2 as well after it implements [[SupportsReportStatistics]]. + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "csv") { + withTempPath { + p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty() + // plan is different, so metric is different + assert(bytesReads.sum === 7864) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } + } + +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala new file mode 100644 index 000000000000..d5885afaee9c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenFileScanSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenFileScanSuite extends FileScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala new file mode 100644 index 000000000000..2421e918bf21 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenGeneratorFunctionSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenGeneratorFunctionSuite extends GeneratorFunctionSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala new file mode 100644 index 000000000000..11b6d99828c6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenInjectRuntimeFilterSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenInjectRuntimeFilterSuite + extends InjectRuntimeFilterSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala new file mode 100644 index 000000000000..0a354a1fc39e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenIntervalFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenIntervalFunctionsSuite extends IntervalFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala new file mode 100644 index 000000000000..1271e43d4840 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJoinSuite.scala @@ -0,0 +1,55 @@ +/* + * 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 + +class GlutenJoinSuite extends JoinSuite with GlutenSQLTestsTrait { + + override def testNameBlackList: Seq[String] = Seq( + // Below tests are to verify operators, just skip. + "join operator selection", + "broadcasted hash join operator selection", + "broadcasted hash outer join operator selection", + "broadcasted existence join operator selection", + "SPARK-28323: PythonUDF should be able to use in join condition", + "SPARK-28345: PythonUDF predicate should be able to pushdown to join", + "cross join with broadcast", + "test SortMergeJoin output ordering", + "SPARK-22445 Respect stream-side child's needCopyResult in BroadcastHashJoin", + "SPARK-32330: Preserve shuffled hash join build side partitioning", + "SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering", + "SPARK-32399: Full outer shuffled hash join", + "SPARK-32649: Optimize BHJ/SHJ inner/semi join with empty hashed relation", + "SPARK-34593: Preserve broadcast nested loop join partitioning and ordering", + "SPARK-35984: Config to force applying shuffled hash join", + "test SortMergeJoin (with spill)", + // NaN is not supported currently, just skip. + "NaN and -0.0 in join keys" + ) + + test(GlutenTestConstants.GLUTEN_TEST + "test case sensitive for BHJ") { + spark.sql("create table t_bhj(a int, b int, C int) using parquet") + spark.sql("insert overwrite t_bhj select id as a, (id+1) as b, (id+2) as c from range(3)") + val sql = + """ + |select /*+ BROADCAST(t1) */ t0.a, t0.b + |from t_bhj as t0 join t_bhj as t1 on t0.a = t1.a and t0.b = t1.b and t0.c = t1.c + |group by t0.a, t0.b + |order by t0.a, t0.b + |""".stripMargin + checkAnswer(spark.sql(sql), Seq(Row(0, 1), Row(1, 2), Row(2, 3))) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala new file mode 100644 index 000000000000..cba4e7a3755d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenJsonFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenJsonFunctionsSuite extends JsonFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala new file mode 100644 index 000000000000..ee39f0138504 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenMathFunctionsSuite extends MathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala new file mode 100644 index 000000000000..d9fc6fd05e1b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMetadataCacheSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenMetadataCacheSuite extends MetadataCacheSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala new file mode 100644 index 000000000000..a95d8a2b2e5c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenMiscFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenMiscFunctionsSuite extends MiscFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala new file mode 100644 index 000000000000..d139221f631a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenNestedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenNestedDataSourceV1Suite extends NestedDataSourceV1Suite with GlutenSQLTestsTrait {} + +class GlutenNestedDataSourceV2Suite extends NestedDataSourceV2Suite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala new file mode 100644 index 000000000000..f8ab9b16adf4 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProcessingTimeSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenProcessingTimeSuite extends ProcessingTimeSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala new file mode 100644 index 000000000000..9cb35efbfbd3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenProductAggSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenProductAggSuite extends ProductAggSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala new file mode 100644 index 000000000000..e345309ab114 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenReplaceNullWithFalseInPredicateEndToEndSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenReplaceNullWithFalseInPredicateEndToEndSuite + extends ReplaceNullWithFalseInPredicateEndToEndSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.scala new file mode 100644 index 000000000000..3ecafb5a19e0 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLInsertTestSuite.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.spark.sql + +import org.apache.spark.SparkConf + +class GlutenFileSourceSQLInsertTestSuite + extends FileSourceSQLInsertTestSuite + with GlutenSQLTestsTrait { + override def sparkConf: SparkConf = { + // Timezone is not supported yet. + super.sparkConf.set("spark.sql.session.timeZone", "UTC") + } +} + +class GlutenDSV2SQLInsertTestSuite extends DSV2SQLInsertTestSuite { + override def sparkConf: SparkConf = { + // Timezone is not supported yet. + super.sparkConf.set("spark.sql.session.timeZone", "UTC") + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala new file mode 100644 index 000000000000..ebb44545ca14 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQuerySuite.scala @@ -0,0 +1,120 @@ +/* + * 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.SparkException +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.internal.SQLConf + +class GlutenSQLQuerySuite extends SQLQuerySuite with GlutenSQLTestsTrait { + import testImplicits._ + + test(GlutenTestConstants.GLUTEN_TEST + "SPARK-28156: self-join should not miss cached view") { + withTable("table1") { + withView("table1_vw") { + withTempView("cachedview") { + val df = Seq.tabulate(5)(x => (x, x + 1, x + 2, x + 3)).toDF("a", "b", "c", "d") + df.write.mode("overwrite").format("parquet").saveAsTable("table1") + sql("drop view if exists table1_vw") + sql("create view table1_vw as select * from table1") + + val cachedView = sql("select a, b, c, d from table1_vw") + + cachedView.createOrReplaceTempView("cachedview") + cachedView.persist() + + val queryDf = sql(s"""select leftside.a, leftside.b + |from cachedview leftside + |join cachedview rightside + |on leftside.a = rightside.a + """.stripMargin) + + val inMemoryTableScan = collect(queryDf.queryExecution.executedPlan) { + case i: InMemoryTableScanExec => i + } + assert(inMemoryTableScan.size == 2) + checkAnswer(queryDf, Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(3, 4) :: Row(4, 5) :: Nil) + } + } + } + + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-33338: GROUP BY using literal map should not fail") { + withTable("t") { + withTempDir { + dir => + sql( + s"CREATE TABLE t USING PARQUET LOCATION '${dir.toURI}' AS SELECT map('k1', 'v1') m," + + s" 'k1' k") + Seq( + "SELECT map('k1', 'v1')[k] FROM t GROUP BY 1", + "SELECT map('k1', 'v1')[k] FROM t GROUP BY map('k1', 'v1')[k]", + "SELECT map('k1', 'v1')[k] a FROM t GROUP BY a" + ).foreach(statement => checkAnswer(sql(statement), Row("v1"))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-33593: Vector reader got incorrect data with binary partition value") { + Seq("false").foreach( + value => { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t1") { + sql("""CREATE TABLE t1(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t1 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t1"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> value) { + withTable("t2") { + sql("""CREATE TABLE t2(name STRING, id BINARY, part BINARY) + |USING PARQUET PARTITIONED BY (part)""".stripMargin) + sql("INSERT INTO t2 PARTITION(part = 'Spark SQL') VALUES('a', X'537061726B2053514C')") + checkAnswer( + sql("SELECT name, cast(id as string), cast(part as string) FROM t2"), + Row("a", "Spark SQL", "Spark SQL")) + } + } + }) + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-33677: LikeSimplification should be skipped if pattern contains any escapeChar") { + withTempView("df") { + Seq("m@ca").toDF("s").createOrReplaceTempView("df") + + val e = intercept[SparkException] { + sql("SELECT s LIKE 'm%@ca' ESCAPE '%' FROM df").collect() + } + assert( + e.getMessage.contains( + "Escape character must be followed by '%', '_' or the escape character itself")) + + checkAnswer(sql("SELECT s LIKE 'm@@ca' ESCAPE '@' FROM df"), Row(true)) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala new file mode 100644 index 000000000000..803010a6c584 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSQLQueryTestSuite.scala @@ -0,0 +1,950 @@ +/* + * 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 io.glutenproject.GlutenConfig +import io.glutenproject.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.SparkConf +import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} +import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND +import org.apache.spark.sql.execution.WholeStageCodegenExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.TimestampTypes +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.Utils + +import java.io.File +import java.net.URI +import java.util.Locale + +import scala.collection.mutable.ArrayBuffer +import scala.sys.process.{Process, ProcessLogger} +import scala.util.Try + +/** + * End-to-end test cases for SQL queries. + * + * Each case is loaded from a file in "spark/sql/core/src/test/resources/sql-tests/inputs". Each + * case has a golden result file in "spark/sql/core/src/test/resources/sql-tests/results". + * + * To run the entire test suite: + * {{{ + * build/sbt "sql/testOnly *SQLQueryTestSuite" + * }}} + * + * To run a single test file upon change: + * {{{ + * build/sbt "~sql/testOnly *SQLQueryTestSuite -- -z inline-table.sql" + * }}} + * + * To re-generate golden files for entire suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite" + * }}} + * + * To re-generate golden file for a single test, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/testOnly *SQLQueryTestSuite -- -z describe.sql" + * }}} + * + * The format for input files is simple: + * 1. A list of SQL queries separated by semicolons by default. If the semicolon cannot + * effectively separate the SQL queries in the test file(e.g. bracketed comments), please use + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END. Lines starting with + * --QUERY-DELIMITER-START and --QUERY-DELIMITER-END represent the beginning and end of a query, + * respectively. Code that is not surrounded by lines that begin with --QUERY-DELIMITER-START and + * --QUERY-DELIMITER-END is still separated by semicolons. 2. Lines starting with -- are treated as + * comments and ignored. 3. Lines starting with --SET are used to specify the configs when running + * this testing file. You can set multiple configs in one --SET, using comma to separate them. Or + * you can use multiple + * --SET statements. 4. Lines starting with --IMPORT are used to load queries from another test + * file. 5. Lines starting with --CONFIG_DIM are used to specify config dimensions of this testing + * file. The dimension name is decided by the string after --CONFIG_DIM. For example, --CONFIG_DIM1 + * belongs to dimension 1. One dimension can have multiple lines, each line representing one config + * set (one or more configs, separated by comma). Spark will run this testing file many times, each + * time picks one config set from each dimension, until all the combinations are tried. For example, + * if dimension 1 has 2 lines, dimension 2 has 3 lines, this testing file will be run 6 times + * (cartesian product). + * + * For example: + * {{{ + * -- this is a comment + * select 1, -1; + * select current_date; + * }}} + * + * The format for golden result files look roughly like: + * {{{ + * -- some header information + * + * -- !query + * select 1, -1 + * -- !query schema + * struct<...schema...> + * -- !query output + * ... data row 1 ... + * ... data row 2 ... + * ... + * + * -- !query + * ... + * }}} + * + * Note that UDF tests work differently. After the test files under 'inputs/udf' directory are + * detected, it creates three test cases: + * + * - Scala UDF test case with a Scalar UDF registered as the name 'udf'. + * + * - Python UDF test case with a Python UDF registered as the name 'udf' iff Python executable and + * pyspark are available. + * + * - Scalar Pandas UDF test case with a Scalar Pandas UDF registered as the name 'udf' iff Python + * executable, pyspark, pandas and pyarrow are available. + * + * Therefore, UDF test cases should have single input and output files but executed by three + * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. + */ +@ExtendedSQLTest +class GlutenSQLQueryTestSuite + extends QueryTest + with SharedSparkSession + with SQLHelper + with SQLQueryTestHelper { + + import IntegratedUDFTestUtils._ + + override protected val regenerateGoldenFiles: Boolean = + System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" + + // FIXME it's not needed to install Spark in testing since the following code only fetchs + // some resource files from source folder + + protected val baseResourcePath = { + // We use a path based on Spark home for 2 reasons: + // 1. Maven can't get correct resource directory when resources in other jars. + // 2. We test subclasses in the hive-thriftserver module. + getWorkspaceFilePath("sql", "core", "src", "test", "resources", "sql-tests").toFile + } + + protected val resourcesPath = { + // We use a path based on Spark home for 2 reasons: + // 1. Maven can't get correct resource directory when resources in other jars. + // 2. We test subclasses in the hive-thriftserver module. + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toFile + } + + protected val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath + protected val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + protected val testDataPath = new File(resourcesPath, "test-data").getAbsolutePath + + protected val validFileExtensions = ".sql" + + /** Test if a command is available. */ + def testCommandAvailable(command: String): Boolean = { + val attempt = if (Utils.isWindows) { + Try(Process(Seq("cmd.exe", "/C", s"where $command")).run(ProcessLogger(_ => ())).exitValue()) + } else { + Try(Process(Seq("sh", "-c", s"command -v $command")).run(ProcessLogger(_ => ())).exitValue()) + } + attempt.isSuccess && attempt.get == 0 + } + + private val isCHBackend = BackendTestUtils.isCHBackendLoaded() + + override protected def sparkConf: SparkConf = { + val conf = super.sparkConf + // Fewer shuffle partitions to speed up testing. + .set(SQLConf.SHUFFLE_PARTITIONS, 4) + // use Java 8 time API to handle negative years properly + .set(SQLConf.DATETIME_JAVA8API_ENABLED, true) + .setAppName("Gluten-UT") + .set("spark.driver.memory", "1G") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.sql.files.maxPartitionBytes", "134217728") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + .set("spark.plugins", "io.glutenproject.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + + if (isCHBackend) { + conf + .set("spark.io.compression.codec", "LZ4") + .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set("spark.gluten.sql.columnar.backend.ch.use.v2", "false") + .set("spark.gluten.sql.enable.native.validation", "false") + .set(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + .set("spark.sql.files.openCostInBytes", "134217728") + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + } else { + conf.set("spark.unsafe.exceptionOnMemoryLeak", "true") + } + conf + } + + // SPARK-32106 Since we add SQL test 'transform.sql' will use `cat` command, + // here we need to ignore it. + private val otherIgnoreList = + if (testCommandAvailable("/bin/bash")) Nil else Set("transform.sql") + + /** List of test cases to ignore, in lower cases. */ + protected def ignoreList: Set[String] = Set( + "ignored.sql", // Do NOT remove this one. It is here to test the ignore functionality. + "explain-aqe.sql", // explain plan is different + "explain-cbo.sql", // explain + "explain.sql", // explain + "group-analytics.sql", // wait velox to fix issue 3357 + "array.sql", // blocked by VELOX-5768 + "higher-order-functions.sql", // blocked by VELOX-5768 + "udf/udf-window.sql", // Local window fixes are not added. + "window.sql", // Local window fixes are not added. + "select_having.sql", // 3.4 failed + "mapconcat.sql" // 3.4 failed + ) ++ otherIgnoreList + + /** + * List of supported cases to run with Velox backend, in lower case. Please add to the supported + * list after enabling a sql test. + */ + + private val veloxSupportedList: Set[String] = Set( +// "bitwise.sql", +// "cast.sql", +// "change-column.sql", +// "charvarchar.sql", +// "columnresolution-negative.sql", + "columnresolution-views.sql", + "columnresolution.sql", +// "comments.sql", + "comparator.sql", +// "count.sql", + "cross-join.sql", +// "csv-functions.sql", +// "cte-legacy.sql", +// "cte-nested.sql", +// "cte-nonlegacy.sql", +// "cte.sql", + "current_database_catalog.sql", +// "date.sql", +// "datetime-formatting-invalid.sql", + // Velox had different handling for some illegal cases. +// "datetime-formatting-legacy.sql", +// "datetime-formatting.sql", +// "datetime-legacy.sql", +// "datetime-parsing-invalid.sql", + "datetime-parsing-legacy.sql", + "datetime-parsing.sql", + "datetime-special.sql", +// "decimalArithmeticOperations.sql", + "describe-part-after-analyze.sql", +// "describe-query.sql", +// "describe-table-after-alter-table.sql", +// "describe-table-column.sql", +// "describe.sql", +// "except-all.sql", +// "except.sql", +// "extract.sql", +// "group-by-filter.sql", +// "group-by-ordinal.sql", +// "group-by.sql", +// "grouping_set.sql", + "having.sql", + "ignored.sql", +// "inline-table.sql", + "inner-join.sql", +// "intersect-all.sql", +// "interval.sql", + "join-empty-relation.sql", +// "join-lateral.sql", +// "json-functions.sql", +// "like-all.sql", +// "like-any.sql", +// "limit.sql", +// "literals.sql", +// "map.sql", + "misc-functions.sql", +// "natural-join.sql", + "null-handling.sql", + "null-propagation.sql", + "operators.sql", + "order-by-nulls-ordering.sql", +// "order-by-ordinal.sql", + "outer-join.sql", + "parse-schema-string.sql", +// "pivot.sql", + "pred-pushdown.sql", + "predicate-functions.sql", +// "query_regex_column.sql", +// "random.sql", +// "regexp-functions.sql", +// "show-create-table.sql", +// "show-tables.sql", +// "show-tblproperties.sql", +// "show-views.sql", +// "show_columns.sql", +// "sql-compatibility-functions.sql", +// "string-functions.sql", + "struct.sql", + "subexp-elimination.sql", +// "table-aliases.sql", +// "table-valued-functions.sql", +// "tablesample-negative.sql", + "subquery/exists-subquery/exists-aggregate.sql", + "subquery/exists-subquery/exists-basic.sql", + "subquery/exists-subquery/exists-cte.sql", + "subquery/exists-subquery/exists-having.sql", + "subquery/exists-subquery/exists-joins-and-set-ops.sql", + "subquery/exists-subquery/exists-orderby-limit.sql", + "subquery/exists-subquery/exists-within-and-or.sql", +// "subquery/in-subquery/in-basic.sql", + "subquery/in-subquery/in-group-by.sql", + "subquery/in-subquery/in-having.sql", + "subquery/in-subquery/in-joins.sql", + "subquery/in-subquery/in-limit.sql", + "subquery/in-subquery/in-multiple-columns.sql", + "subquery/in-subquery/in-order-by.sql", + "subquery/in-subquery/in-set-operations.sql", + "subquery/in-subquery/in-with-cte.sql", + "subquery/in-subquery/nested-not-in.sql", + "subquery/in-subquery/not-in-group-by.sql", + "subquery/in-subquery/not-in-joins.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column.sql", + "subquery/in-subquery/not-in-unit-tests-multi-column-literal.sql", + "subquery/in-subquery/not-in-unit-tests-single-column.sql", + "subquery/in-subquery/not-in-unit-tests-single-column-literal.sql", + "subquery/in-subquery/simple-in.sql", +// "subquery/negative-cases/invalid-correlation.sql", +// "subquery/negative-cases/subq-input-typecheck.sql", + "subquery/scalar-subquery/scalar-subquery-predicate.sql", + "subquery/scalar-subquery/scalar-subquery-select.sql", + "subquery/subquery-in-from.sql", +// "postgreSQL/aggregates_part1.sql", + "postgreSQL/aggregates_part2.sql", +// "postgreSQL/aggregates_part3.sql", +// "postgreSQL/aggregates_part4.sql", +// "postgreSQL/boolean.sql", + "postgreSQL/case.sql", + "postgreSQL/comments.sql", +// "postgreSQL/create_view.sql", +// "postgreSQL/date.sql", +// "postgreSQL/float4.sql", + "postgreSQL/insert.sql", + "postgreSQL/int2.sql", +// "postgreSQL/int4.sql", +// "postgreSQL/int8.sql", +// "postgreSQL/interval.sql", +// "postgreSQL/join.sql", +// "postgreSQL/limit.sql", +// "postgreSQL/numeric.sql", + "postgreSQL/select.sql", + "postgreSQL/select_distinct.sql", +// "postgreSQL/select_having.sql", +// "postgreSQL/select_implicit.sql", +// "postgreSQL/strings.sql", +// "postgreSQL/text.sql", + "postgreSQL/timestamp.sql", +// "postgreSQL/union.sql", + "postgreSQL/window_part1.sql", +// "postgreSQL/window_part2.sql", +// "postgreSQL/window_part3.sql", +// "postgreSQL/window_part4.sql", +// "postgreSQL/with.sql", + "datetime-special.sql", +// "timestamp-ansi.sql", +// "timestamp.sql", + "arrayJoin.sql", + "binaryComparison.sql", +// "booleanEquality.sql", +// "caseWhenCoercion.sql", + "concat.sql", +// "dateTimeOperations.sql", +// "decimalPrecision.sql", +// "division.sql", + "elt.sql", +// "ifCoercion.sql", + "implicitTypeCasts.sql", +// "inConversion.sql", +// "mapZipWith.sql", +// "mapconcat.sql", +// "promoteStrings.sql", +// "stringCastAndExpressions.sql", +// "widenSetOperationTypes.sql", +// "windowFrameCoercion.sql", + "timestamp-ltz.sql", +// "timestamp-ntz.sql", +// "timezone.sql", +// "transform.sql", +// "try_arithmetic.sql", + "try_cast.sql", +// "udaf.sql", +// "union.sql", + "using-join.sql", + "window.sql", + "udf-union.sql", + "udf-window.sql" + ) + + /** + * List of supported cases to run with Clickhouse backend, in lower case. Please add to the + * supported list after enabling a sql test. + */ + private val CHSupportedList: Set[String] = Set() + + // List of supported cases to run with a certain backend, in lower case. + private val supportedList: Set[String] = if (isCHBackend) { + CHSupportedList + } else { + veloxSupportedList + } + // Create all the test cases. + listTestCases.foreach(createScalaTestCase) + + /** A single SQL query's output. */ + protected case class QueryOutput(sql: String, schema: String, output: String) { + override def toString: String = { + // We are explicitly not using multi-line string due to stripMargin removing "|" in output. + s"-- !query\n" + + sql + "\n" + + s"-- !query schema\n" + + schema + "\n" + + s"-- !query output\n" + + output + } + } + + /** A test case. */ + protected trait TestCase { + val name: String + val inputFile: String + val resultFile: String + } + + /** + * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, + * PgSQL tests require to register some UDF functions. + */ + protected trait PgSQLTest + + /** traits that indicate ANSI-related tests with the ANSI mode enabled. */ + protected trait AnsiTest + + /** traits that indicate the default timestamp type is TimestampNTZType. */ + protected trait TimestampNTZTest + + protected trait UDFTest { + val udf: TestUDF + } + + /** A regular test case. */ + protected case class RegularTestCase(name: String, inputFile: String, resultFile: String) + extends TestCase + + /** A PostgreSQL test case. */ + protected case class PgSQLTestCase(name: String, inputFile: String, resultFile: String) + extends TestCase + with PgSQLTest + + /** A UDF test case. */ + protected case class UDFTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) + extends TestCase + with UDFTest + + /** A UDF PostgreSQL test case. */ + protected case class UDFPgSQLTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) + extends TestCase + with UDFTest + with PgSQLTest + + /** An ANSI-related test case. */ + protected case class AnsiTestCase(name: String, inputFile: String, resultFile: String) + extends TestCase + with AnsiTest + + /** An date time test case with default timestamp as TimestampNTZType */ + protected case class TimestampNTZTestCase(name: String, inputFile: String, resultFile: String) + extends TestCase + with TimestampNTZTest + + protected def createScalaTestCase(testCase: TestCase): Unit = { + // If a test case is not in the test list, or it is in the ignore list, ignore this test case. + if ( + !supportedList.exists( + t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT))) || + ignoreList.exists( + t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT))) + ) { + // Create a test case to ignore this case. + ignore(testCase.name) { /* Do nothing */ } + } else { + testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && !shouldTestPythonUDFs => + ignore( + s"${testCase.name} is skipped because " + + s"[$pythonExec] and/or pyspark were not available.") { + /* Do nothing */ + } + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && !shouldTestPandasUDFs => + ignore( + s"${testCase.name} is skipped because pyspark," + + s"pandas and/or pyarrow were not available in [$pythonExec].") { + /* Do nothing */ + } + case _ => + // Create a test case to run this case. + test(testCase.name) { + runTest(testCase) + } + } + } + } + + /** Run a test case. */ + protected def runTest(testCase: TestCase): Unit = { + def splitWithSemicolon(seq: Seq[String]) = { + seq.mkString("\n").split("(?<=[^\\\\]);") + } + + def splitCommentsAndCodes(input: String) = input.split("\n").partition { + line => + val newLine = line.trim + newLine.startsWith("--") && !newLine.startsWith("--QUERY-DELIMITER") + } + + val input = fileToString(new File(testCase.inputFile)) + + val (comments, code) = splitCommentsAndCodes(input) + + // If `--IMPORT` found, load code from another test case file, then insert them + // into the head in this test. + val importedTestCaseName = comments.filter(_.startsWith("--IMPORT ")).map(_.substring(9)) + val importedCode = importedTestCaseName.flatMap { + testCaseName => + listTestCases.find(_.name == testCaseName).map { + testCase => + val input = fileToString(new File(testCase.inputFile)) + val (_, code) = splitCommentsAndCodes(input) + code + } + }.flatten + + val allCode = importedCode ++ code + val tempQueries = if (allCode.exists(_.trim.startsWith("--QUERY-DELIMITER"))) { + // Although the loop is heavy, only used for bracketed comments test. + val queries = new ArrayBuffer[String] + val otherCodes = new ArrayBuffer[String] + var tempStr = "" + var start = false + for (c <- allCode) { + if (c.trim.startsWith("--QUERY-DELIMITER-START")) { + start = true + queries ++= splitWithSemicolon(otherCodes.toSeq) + otherCodes.clear() + } else if (c.trim.startsWith("--QUERY-DELIMITER-END")) { + start = false + queries += s"\n${tempStr.stripSuffix(";")}" + tempStr = "" + } else if (start) { + tempStr += s"\n$c" + } else { + otherCodes += c + } + } + if (otherCodes.nonEmpty) { + queries ++= splitWithSemicolon(otherCodes.toSeq) + } + queries.toSeq + } else { + splitWithSemicolon(allCode).toSeq + } + + // List of SQL queries to run + val queries = tempQueries + .map(_.trim) + .filter(_ != "") + .toSeq + // Fix misplacement when comment is at the end of the query. + .map(_.split("\n").filterNot(_.startsWith("--")).mkString("\n")) + .map(_.trim) + .filter(_ != "") + + val settingLines = comments.filter(_.startsWith("--SET ")).map(_.substring(6)) + val settings = settingLines.flatMap(_.split(",").map { + kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + + if (regenerateGoldenFiles) { + runQueries(queries, testCase, settings) + } else { + // A config dimension has multiple config sets, and a config set has multiple configs. + // - config dim: Seq[Seq[(String, String)]] + // - config set: Seq[(String, String)] + // - config: (String, String)) + // We need to do cartesian product for all the config dimensions, to get a list of + // config sets, and run the query once for each config set. + val configDimLines = comments.filter(_.startsWith("--CONFIG_DIM")).map(_.substring(12)) + val configDims = configDimLines.groupBy(_.takeWhile(_ != ' ')).mapValues { + lines => + lines + .map(_.dropWhile(_ != ' ').substring(1)) + .map(_.split(",") + .map { + kv => + val (conf, value) = kv.span(_ != '=') + conf.trim -> value.substring(1).trim + } + .toSeq) + .toSeq + } + + val configSets = configDims.values.foldLeft(Seq(Seq[(String, String)]())) { + (res, dim) => dim.flatMap(configSet => res.map(_ ++ configSet)) + } + + configSets.foreach { + configSet => + try { + runQueries(queries, testCase, settings ++ configSet) + } catch { + case e: Throwable => + val configs = configSet.map { case (k, v) => s"$k=$v" } + logError(s"Error using configs: ${configs.mkString(",")}") + throw e + } + } + } + } + + protected def runQueries( + queries: Seq[String], + testCase: TestCase, + configSet: Seq[(String, String)]): Unit = { + // Create a local SparkSession to have stronger isolation between different test cases. + // This does not isolate catalog changes. + val localSparkSession = spark.newSession() + + testCase match { + case udfTestCase: UDFTest => + registerTestUDF(udfTestCase.udf, localSparkSession) + case _ => + } + + testCase match { + case _: PgSQLTest => + // booleq/boolne used by boolean.sql + localSparkSession.udf.register("booleq", (b1: Boolean, b2: Boolean) => b1 == b2) + localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) + // vol used by boolean.sql and case.sql. + localSparkSession.udf.register("vol", (s: String) => s) + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.LEGACY_INTERVAL_ENABLED.key, true) + case _: AnsiTest => + localSparkSession.conf.set(SQLConf.ANSI_ENABLED.key, true) + case _: TimestampNTZTest => + localSparkSession.conf.set( + SQLConf.TIMESTAMP_TYPE.key, + TimestampTypes.TIMESTAMP_NTZ.toString) + case _ => + } + + if (configSet.nonEmpty) { + // Execute the list of set operation in order to add the desired configs + val setOperations = configSet.map { case (key, value) => s"set $key=$value" } + logInfo(s"Setting configs: ${setOperations.mkString(", ")}") + setOperations.foreach(localSparkSession.sql) + } + + // Run the SQL queries preparing them for comparison. + val outputs: Seq[QueryOutput] = queries.map { + sql => + val (schema, output) = handleExceptions(getNormalizedResult(localSparkSession, sql)) + // We might need to do some query canonicalization in the future. + QueryOutput( + sql = sql, + schema = schema, + output = output.mkString("\n").replaceAll("\\s+$", "")) + } + + if (regenerateGoldenFiles) { + // Again, we are explicitly not using multi-line string due to stripMargin removing "|". + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n" + + s"-- Number of queries: ${outputs.size}\n\n\n" + + outputs.mkString("\n\n\n") + "\n" + } + val resultFile = new File(testCase.resultFile) + val parent = resultFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(resultFile, goldenOutput) + } + + // This is a temporary workaround for SPARK-28894. The test names are truncated after + // the last dot due to a bug in SBT. This makes easier to debug via Jenkins test result + // report. See SPARK-28894. + // See also SPARK-29127. It is difficult to see the version information in the failed test + // cases so the version information related to Python was also added. + val clue = testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && shouldTestPythonUDFs => + s"${testCase.name}${System.lineSeparator()}Python: $pythonVer${System.lineSeparator()}" + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && shouldTestPandasUDFs => + s"${testCase.name}${System.lineSeparator()}" + + s"Python: $pythonVer Pandas: $pandasVer PyArrow: $pyarrowVer${System.lineSeparator()}" + case _ => + s"${testCase.name}${System.lineSeparator()}" + } + + withClue(clue) { + // Read back the golden file. + val expectedOutputs: Seq[QueryOutput] = { + val goldenOutput = fileToString(new File(testCase.resultFile)) + val segments = goldenOutput.split("-- !query.*\n") + + // each query has 3 segments, plus the header + assert( + segments.size == outputs.size * 3 + 1, + s"Expected ${outputs.size * 3 + 1} blocks in result file but got ${segments.size}. " + + s"Try regenerate the result files.") + Seq.tabulate(outputs.size) { + i => + QueryOutput( + sql = segments(i * 3 + 1).trim, + schema = segments(i * 3 + 2).trim, + output = segments(i * 3 + 3).replaceAll("\\s+$", "") + ) + } + } + + // Compare results. + assertResult(expectedOutputs.size, s"Number of queries should be ${expectedOutputs.size}") { + outputs.size + } + + outputs.zip(expectedOutputs).zipWithIndex.foreach { + case ((output, expected), i) => + assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { + output.sql + } + assertResult( + expected.schema, + s"Schema did not match for query #$i\n${expected.sql}: $output") { + output.schema + } + assertResult( + expected.output, + s"Result did not match" + + s" for query #$i\n${expected.sql}")(output.output) + } + } + } + + protected lazy val listTestCases: Seq[TestCase] = { + listFilesRecursively(new File(inputFilePath)).flatMap { + file => + val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" + val absPath = file.getAbsolutePath + val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) + + if ( + file.getAbsolutePath.startsWith( + s"$inputFilePath${File.separator}udf${File.separator}postgreSQL") + ) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { + udf => UDFPgSQLTestCase(s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { + udf => UDFTestCase(s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}postgreSQL")) { + PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}ansi")) { + AnsiTestCase(testCaseName, absPath, resultFile) :: Nil + } else if ( + file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}timestampNTZ") + ) { + TimestampNTZTestCase(testCaseName, absPath, resultFile) :: Nil + } else { + RegularTestCase(testCaseName, absPath, resultFile) :: Nil + } + } + } + + /** Returns all the files (not directories) in a directory, recursively. */ + protected def listFilesRecursively(path: File): Seq[File] = { + val (dirs, files) = path.listFiles().partition(_.isDirectory) + // Filter out test files with invalid extensions such as temp files created + // by vi (.swp), Mac (.DS_Store) etc. + val filteredFiles = files.filter(_.getName.endsWith(validFileExtensions)) + filteredFiles ++ dirs.flatMap(listFilesRecursively) + } + + /** Load built-in test tables into the SparkSession. */ + private def createTestTables(session: SparkSession): Unit = { + import session.implicits._ + + // Before creating test tables, deletes orphan directories in warehouse dir + Seq("testdata", "arraydata", "mapdata", "aggtest", "onek", "tenk1").foreach { + dirName => + val f = new File(new URI(s"${conf.warehousePath}/$dirName")) + if (f.exists()) { + Utils.deleteRecursively(f) + } + } + + (1 to 100) + .map(i => (i, i.toString)) + .toDF("key", "value") + .repartition(1) + .write + .format("parquet") + .saveAsTable("testdata") + + ((Seq(1, 2, 3), Seq(Seq(1, 2, 3))) :: (Seq(2, 3, 4), Seq(Seq(2, 3, 4))) :: Nil) + .toDF("arraycol", "nestedarraycol") + .write + .format("parquet") + .saveAsTable("arraydata") + + (Tuple1(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: + Tuple1(Map(1 -> "a2", 2 -> "b2", 3 -> "c2", 4 -> "d2")) :: + Tuple1(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: + Tuple1(Map(1 -> "a4", 2 -> "b4")) :: + Tuple1(Map(1 -> "a5")) :: Nil) + .toDF("mapcol") + .write + .format("parquet") + .saveAsTable("mapdata") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema("a int, b float") + .load(testDataPath + "/postgresql/agg.data") + .write + .format("parquet") + .saveAsTable("aggtest") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema(""" + |unique1 int, + |unique2 int, + |two int, + |four int, + |ten int, + |twenty int, + |hundred int, + |thousand int, + |twothousand int, + |fivethous int, + |tenthous int, + |odd int, + |even int, + |stringu1 string, + |stringu2 string, + |string4 string + """.stripMargin) + .load(testDataPath + "/postgresql/onek.data") + .write + .format("parquet") + .saveAsTable("onek") + + session.read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema(""" + |unique1 int, + |unique2 int, + |two int, + |four int, + |ten int, + |twenty int, + |hundred int, + |thousand int, + |twothousand int, + |fivethous int, + |tenthous int, + |odd int, + |even int, + |stringu1 string, + |stringu2 string, + |string4 string + """.stripMargin) + .load(testDataPath + "/postgresql/tenk.data") + .write + .format("parquet") + .saveAsTable("tenk1") + } + + private def removeTestTables(session: SparkSession): Unit = { + session.sql("DROP TABLE IF EXISTS testdata") + session.sql("DROP TABLE IF EXISTS arraydata") + session.sql("DROP TABLE IF EXISTS mapdata") + session.sql("DROP TABLE IF EXISTS aggtest") + session.sql("DROP TABLE IF EXISTS onek") + session.sql("DROP TABLE IF EXISTS tenk1") + } + + override def beforeAll(): Unit = { + super.beforeAll() + createTestTables(spark) + RuleExecutor.resetMetrics() + CodeGenerator.resetCompileTime() + WholeStageCodegenExec.resetCodeGenTime() + } + + override def afterAll(): Unit = { + try { + removeTestTables(spark) + + // For debugging dump some statistics about how much time was spent in various optimizer rules + logWarning(RuleExecutor.dumpTimeSpent()) + + val codeGenTime = WholeStageCodegenExec.codeGenTime.toDouble / NANOS_PER_SECOND + val compileTime = CodeGenerator.compileTime.toDouble / NANOS_PER_SECOND + val codegenInfo = + s""" + |=== Metrics of Whole-stage Codegen === + |Total code generation time: $codeGenTime seconds + |Total compile time: $compileTime seconds + """.stripMargin + logWarning(codegenInfo) + } finally { + super.afterAll() + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala new file mode 100644 index 000000000000..75bc845b5c8f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenScalaReflectionRelationSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenScalaReflectionRelationSuite + extends ScalaReflectionRelationSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala new file mode 100644 index 000000000000..569de43a75ca --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSerializationSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenSerializationSuite extends SerializationSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala new file mode 100644 index 000000000000..29ff39efed2d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStatisticsCollectionSuite.scala @@ -0,0 +1,78 @@ +/* + * 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.ColumnStat +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC +import org.apache.spark.sql.functions.timestamp_seconds +import org.apache.spark.sql.types.{DataType, DateType, TimestampType} + +import java.util.TimeZone +import java.util.concurrent.TimeUnit + +class GlutenStatisticsCollectionSuite extends StatisticsCollectionSuite with GlutenSQLTestsTrait { + + import testImplicits._ + + test( + GlutenTestConstants.GLUTEN_TEST + + "store and retrieve column stats in different time zones") { + // TODO: bug fix on TableScan. + // val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) + val (start, end) = (0, 200) + + def checkTimestampStats(t: DataType, srcTimeZone: TimeZone, dstTimeZone: TimeZone)( + checker: ColumnStat => Unit): Unit = { + val table = "time_table" + val column = "T" + val original = TimeZone.getDefault + try { + withTable(table) { + TimeZone.setDefault(srcTimeZone) + spark + .range(start, end) + .select(timestamp_seconds($"id").cast(t).as(column)) + .write + .saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") + + TimeZone.setDefault(dstTimeZone) + val stats = getCatalogTable(table).stats.get.colStats(column).toPlanStat(column, t) + checker(stats) + } + } finally { + TimeZone.setDefault(original) + } + } + + DateTimeTestUtils.outstandingZoneIds.foreach { + zid => + val timeZone = TimeZone.getTimeZone(zid) + checkTimestampStats(DateType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(start)) + assert(stats.max.get.asInstanceOf[Int] == TimeUnit.SECONDS.toDays(end - 1)) + } + checkTimestampStats(TimestampType, TimeZoneUTC, timeZone) { + stats => + assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) + assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala new file mode 100644 index 000000000000..054be9e87f38 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenStringFunctionsSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import io.glutenproject.utils.FallbackUtil + +import org.apache.spark.sql.catalyst.expressions.ExpressionEvalHelper +import org.apache.spark.sql.functions._ + +import org.junit.Assert + +class GlutenStringFunctionsSuite + extends StringFunctionsSuite + with GlutenSQLTestsTrait + with ExpressionEvalHelper { + + import testImplicits._ + + override def testNameBlackList: Seq[String] = super.testNameBlackList ++ Seq( + "string / binary length function" + ) + + test(GlutenTestConstants.GLUTEN_TEST + "string split function with no limit and regex pattern") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A")) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_")) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + test(GlutenTestConstants.GLUTEN_TEST + "string split function with limit explicitly set to 0") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", 0)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", 0)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } + + test(GlutenTestConstants.GLUTEN_TEST + "string split function with negative limit") { + val df1 = Seq(("aaAbbAcc4")).toDF("a").select(split($"a", "A", -1)) + checkAnswer(df1, Row(Seq("aa", "bb", "cc4"))) + Assert.assertFalse(FallbackUtil.hasFallback(df1.queryExecution.executedPlan)) + + // scalastyle:off nonascii + val df2 = Seq(("test_gluten单测_")).toDF("a").select(split($"a", "_", -2)) + checkAnswer(df2, Row(Seq("test", "gluten单测", ""))) + // scalastyle:on nonascii + Assert.assertFalse(FallbackUtil.hasFallback(df2.queryExecution.executedPlan)) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala new file mode 100644 index 000000000000..6251397f51b5 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenSubquerySuite.scala @@ -0,0 +1,59 @@ +/* + * 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 io.glutenproject.execution.{FileSourceScanExecTransformer, WholeStageTransformer} + +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST + +class GlutenSubquerySuite extends SubquerySuite with GlutenSQLTestsTrait { + + // Test Canceled: IntegratedUDFTestUtils.shouldTestPythonUDFs was false + override def testNameBlackList: Seq[String] = Seq( + "SPARK-28441: COUNT bug in WHERE clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug in SELECT clause (Project) with PythonUDF", + "SPARK-28441: COUNT bug in Aggregate with PythonUDF", + "SPARK-28441: COUNT bug negative examples with PythonUDF", + "SPARK-28441: COUNT bug in nested subquery with PythonUDF", + "SPARK-28441: COUNT bug with nasty predicate expr with PythonUDF", + "SPARK-28441: COUNT bug in HAVING clause (Filter) with PythonUDF", + "SPARK-28441: COUNT bug with attribute ref in subquery input and output with PythonUDF" + ) + + // === Following cases override super class's cases === + + test( + GLUTEN_TEST + + "SPARK-26893 Allow pushdown of partition pruning subquery filters to file source") { + withTable("a", "b") { + spark.range(4).selectExpr("id", "id % 2 AS p").write.partitionBy("p").saveAsTable("a") + spark.range(2).write.saveAsTable("b") + + // need to execute the query before we can examine fs.inputRDDs() + val df = sql("SELECT * FROM a WHERE p <= (SELECT MIN(id) FROM b)") + checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) + assert(stripAQEPlan(df.queryExecution.executedPlan).collectFirst { + case t: WholeStageTransformer => t + } match { + case Some(WholeStageTransformer(fs: FileSourceScanExecTransformer, _)) => + fs.dynamicallySelectedPartitions + .exists(_.files.exists(_.getPath.toString.contains("p=0"))) + case _ => false + }) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala new file mode 100644 index 000000000000..cff309cfce2b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenTypedImperativeAggregateSuite.scala @@ -0,0 +1,21 @@ +/* + * 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 + +class GlutenTypedImperativeAggregateSuite + extends TypedImperativeAggregateSuite + with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala new file mode 100644 index 000000000000..1a2f38638cf3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenUnwrapCastInComparisonEndToEndSuite.scala @@ -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.spark.sql + +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST + +import scala.collection.immutable.Seq + +class GlutenUnwrapCastInComparisonEndToEndSuite + extends UnwrapCastInComparisonEndToEndSuite + with GlutenSQLTestsTrait { + + import testImplicits._ + + test(GLUTEN_TEST + "cases when literal is max") { + withTable(t) { + Seq[(Integer, java.lang.Short, java.lang.Float)]( + (1, 100.toShort, 3.14.toFloat), + (2, Short.MaxValue, Float.NaN), + (3, Short.MinValue, Float.PositiveInfinity), + (4, 0.toShort, Float.MaxValue), + (5, null, null)) + .toDF("c1", "c2", "c3") + .write + .saveAsTable(t) + val df = spark.table(t) + + val lit = Short.MaxValue.toInt + checkAnswer(df.where(s"c2 > $lit").select("c1"), Seq.empty) + checkAnswer(df.where(s"c2 >= $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 == $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 <=> $lit").select("c1"), Row(2)) + checkAnswer(df.where(s"c2 != $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 <= $lit").select("c1"), Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) + checkAnswer(df.where(s"c2 < $lit").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + + // NaN is not supported in velox, so unexpected result will be obtained. +// checkAnswer(df.where(s"c3 > double('nan')").select("c1"), Seq.empty) +// checkAnswer(df.where(s"c3 >= double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 == double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 <=> double('nan')").select("c1"), Row(2)) +// checkAnswer(df.where(s"c3 != double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) +// checkAnswer(df.where(s"c3 <= double('nan')").select("c1"), +// Row(1) :: Row(2) :: Row(3) :: Row(4) :: Nil) +// checkAnswer(df.where(s"c3 < double('nan')").select("c1"), Row(1) :: Row(3) :: Row(4) :: Nil) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala new file mode 100644 index 000000000000..918a96c49e30 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/GlutenXPathFunctionsSuite.scala @@ -0,0 +1,19 @@ +/* + * 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 + +class GlutenXPathFunctionsSuite extends XPathFunctionsSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala new file mode 100644 index 000000000000..14079037518f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenArithmeticExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenArithmeticExpressionSuite extends ArithmeticExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala new file mode 100644 index 000000000000..fd9827ddf502 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenBitwiseExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenBitwiseExpressionsSuite extends BitwiseExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala new file mode 100644 index 000000000000..6d330cf02597 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCastSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.types._ + +import java.sql.Date + +class GlutenCastSuite extends CastSuiteBase with GlutenTestsTrait { + override def cast(v: Any, targetType: DataType, timeZoneId: Option[String] = None): Cast = { + v match { + case lit: Expression => + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + case _ => + val lit = Literal(v) + logDebug(s"Cast from: ${lit.dataType.typeName}, to: ${targetType.typeName}") + Cast(lit, targetType, timeZoneId) + } + } + + // Register UDT For test("SPARK-32828") + UDTRegistration.register(classOf[IExampleBaseType].getName, classOf[ExampleBaseTypeUDT].getName) + UDTRegistration.register(classOf[IExampleSubType].getName, classOf[ExampleSubTypeUDT].getName) + + test("missing cases - from boolean") { + (DataTypeTestUtils.numericTypeWithoutDecimal + BooleanType).foreach { + t => + t match { + case BooleanType => + checkEvaluation(cast(cast(true, BooleanType), t), true) + checkEvaluation(cast(cast(false, BooleanType), t), false) + case _ => + checkEvaluation(cast(cast(true, BooleanType), t), 1) + checkEvaluation(cast(cast(false, BooleanType), t), 0) + } + } + } + + test("missing cases - from byte") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ByteType), t), 0) + checkEvaluation(cast(cast(-1, ByteType), t), -1) + checkEvaluation(cast(cast(1, ByteType), t), 1) + } + } + + test("missing cases - from short") { + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { + t => + checkEvaluation(cast(cast(0, ShortType), t), 0) + checkEvaluation(cast(cast(-1, ShortType), t), -1) + checkEvaluation(cast(cast(1, ShortType), t), 1) + } + } + + test("missing cases - date self check") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(cast(d, DateType), d) + } + + override protected def evalMode: EvalMode.Value = EvalMode.LEGACY +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.scala new file mode 100644 index 000000000000..f223cf7b8e90 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenCollectionExpressionsSuite.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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.types._ + +class GlutenCollectionExpressionsSuite extends CollectionExpressionsSuite with GlutenTestsTrait { + test(GLUTEN_TEST + "Concat") { + // Primitive-type elements + val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) + val ai1 = Literal.create(Seq.empty[Integer], ArrayType(IntegerType, containsNull = false)) + val ai2 = Literal.create(Seq(4, null, 5), ArrayType(IntegerType, containsNull = true)) + val ai3 = Literal.create(Seq(null, null), ArrayType(IntegerType, containsNull = true)) + val ai4 = Literal.create(null, ArrayType(IntegerType, containsNull = false)) + + // checkEvaluation(Concat(Seq(ai0)), Seq(1, 2, 3)) + checkEvaluation(Concat(Seq(ai0, ai1)), Seq(1, 2, 3)) + checkEvaluation(Concat(Seq(ai1, ai0)), Seq(1, 2, 3)) + checkEvaluation(Concat(Seq(ai0, ai0)), Seq(1, 2, 3, 1, 2, 3)) + checkEvaluation(Concat(Seq(ai0, ai2)), Seq(1, 2, 3, 4, null, 5)) + checkEvaluation(Concat(Seq(ai0, ai3, ai2)), Seq(1, 2, 3, null, null, 4, null, 5)) + checkEvaluation(Concat(Seq(ai4)), null) + checkEvaluation(Concat(Seq(ai0, ai4)), null) + checkEvaluation(Concat(Seq(ai4, ai0)), null) + + // Non-primitive-type elements + val as0 = Literal.create(Seq("a", "b", "c"), ArrayType(StringType, containsNull = false)) + val as1 = Literal.create(Seq.empty[String], ArrayType(StringType, containsNull = false)) + val as2 = Literal.create(Seq("d", null, "e"), ArrayType(StringType, containsNull = true)) + val as3 = Literal.create(Seq(null, null), ArrayType(StringType, containsNull = true)) + val as4 = Literal.create(null, ArrayType(StringType, containsNull = false)) + + val aa0 = Literal.create( + Seq(Seq("a", "b"), Seq("c")), + ArrayType(ArrayType(StringType, containsNull = false), containsNull = false)) + val aa1 = Literal.create( + Seq(Seq("d"), Seq("e", "f")), + ArrayType(ArrayType(StringType, containsNull = false), containsNull = false)) + val aa2 = Literal.create( + Seq(Seq("g", null), null), + ArrayType(ArrayType(StringType, containsNull = true), containsNull = true)) + + // checkEvaluation(Concat(Seq(as0)), Seq("a", "b", "c")) + checkEvaluation(Concat(Seq(as0, as1)), Seq("a", "b", "c")) + checkEvaluation(Concat(Seq(as1, as0)), Seq("a", "b", "c")) + checkEvaluation(Concat(Seq(as0, as0)), Seq("a", "b", "c", "a", "b", "c")) + checkEvaluation(Concat(Seq(as0, as2)), Seq("a", "b", "c", "d", null, "e")) + checkEvaluation(Concat(Seq(as0, as3, as2)), Seq("a", "b", "c", null, null, "d", null, "e")) + checkEvaluation(Concat(Seq(as4)), null) + checkEvaluation(Concat(Seq(as0, as4)), null) + checkEvaluation(Concat(Seq(as4, as0)), null) + + checkEvaluation(Concat(Seq(aa0, aa1)), Seq(Seq("a", "b"), Seq("c"), Seq("d"), Seq("e", "f"))) + + assert(Concat(Seq(ai0, ai1)).dataType.asInstanceOf[ArrayType].containsNull === false) + assert(Concat(Seq(ai0, ai2)).dataType.asInstanceOf[ArrayType].containsNull) + assert(Concat(Seq(as0, as1)).dataType.asInstanceOf[ArrayType].containsNull === false) + assert(Concat(Seq(as0, as2)).dataType.asInstanceOf[ArrayType].containsNull) + assert( + Concat(Seq(aa0, aa1)).dataType === + ArrayType(ArrayType(StringType, containsNull = false), containsNull = false)) + assert( + Concat(Seq(aa0, aa2)).dataType === + ArrayType(ArrayType(StringType, containsNull = true), containsNull = true)) + + // force split expressions for input in generated code + checkEvaluation(Concat(Seq.fill(100)(ai0)), Seq.fill(100)(Seq(1, 2, 3)).flatten) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala new file mode 100644 index 000000000000..f5f278361e1f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenComplexTypeSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenComplexTypeSuite extends ComplexTypeSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala new file mode 100644 index 000000000000..923f5f87bcc2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenConditionalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenConditionalExpressionSuite extends ConditionalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala new file mode 100644 index 000000000000..b599a277214b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDateExpressionsSuite.scala @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, TimeZoneUTC} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.unsafe.types.UTF8String + +import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat +import java.time.ZoneId +import java.util.{Locale, TimeZone} +import java.util.concurrent.TimeUnit._ + +class GlutenDateExpressionsSuite extends DateExpressionsSuite with GlutenTestsTrait { + override def testIntegralInput(testFunc: Number => Unit): Unit = { + def checkResult(input: Long): Unit = { + if (input.toByte == input) { + testFunc(input.toByte) + } else if (input.toShort == input) { + testFunc(input.toShort) + } else if (input.toInt == input) { + testFunc(input.toInt) + } else { + testFunc(input) + } + } + + checkResult(0) + checkResult(Byte.MaxValue) + checkResult(Byte.MinValue) + checkResult(Short.MaxValue) + checkResult(Short.MinValue) + // Spark collect causes integer overflow. + // checkResult(Int.MaxValue) + // checkResult(Int.MinValue) + // checkResult(Int.MaxValue.toLong + 100) + // checkResult(Int.MinValue.toLong - 100) + } + + test(GLUTEN_TEST + "TIMESTAMP_MICROS") { + def testIntegralFunc(value: Number): Unit = { + checkEvaluation(MicrosToTimestamp(Literal(value)), value.longValue()) + } + + // test null input + checkEvaluation(MicrosToTimestamp(Literal(null, IntegerType)), null) + + // test integral input + testIntegralInput(testIntegralFunc) + // test max/min input + // Spark collect causes long overflow. + // testIntegralFunc(Long.MaxValue) + // testIntegralFunc(Long.MinValue) + } + + val outstandingTimezonesIds: Seq[String] = Seq( + // Velox doesn't support timezones like UTC. + // "UTC", + PST.getId, + CET.getId, + "Africa/Dakar", + LA.getId, + "Asia/Urumqi", + "Asia/Hong_Kong", + "Europe/Brussels") + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(getZoneId) + + test(GLUTEN_TEST + "unix_timestamp") { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(0))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 0L) + checkEvaluation( + UnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal(new Timestamp(1000000)), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + 1000L) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + checkEvaluation( + UnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + UnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), + tz.toZoneId)) + ) + val t1 = UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + .eval() + .asInstanceOf[Long] + val t2 = UnixTimestamp(CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")) + .eval() + .asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp( + Literal.create(null, DateType), + Literal("yyyy-MM-dd HH:mm:ss"), + timeZoneId), + null) + checkEvaluation( + UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), tz.toZoneId)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + UnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + test(GLUTEN_TEST + "to_unix_timestamp") { + withDefaultTimeZone(UTC) { + for (zid <- outstandingZoneIds) { + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> zid.getId + ) { + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) + val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" + val sdf2 = new SimpleDateFormat(fmt2, Locale.US) + val fmt3 = "yy-MM-dd" + val sdf3 = new SimpleDateFormat(fmt3, Locale.US) + sdf3.setTimeZone(TimeZoneUTC) + + val timeZoneId = Option(zid.getId) + val tz = TimeZone.getTimeZone(zid) + sdf1.setTimeZone(tz) + sdf2.setTimeZone(tz) + + val date1 = Date.valueOf("2015-07-24") + checkEvaluation( + ToUnixTimestamp(Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), + 0L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf1.format(new Timestamp(1000000))), + Literal(fmt1), + timeZoneId), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(new Timestamp(1000000)), Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp( + Literal( + DateTimeUtils.microsToLocalDateTime(DateTimeUtils.millisToMicros(1000000))), + Literal(fmt1)), + 1000L) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid))) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf2.format(new Timestamp(-1000000))), + Literal(fmt2), + timeZoneId), + -1000L) + checkEvaluation( + ToUnixTimestamp( + Literal(sdf3.format(Date.valueOf("2015-07-24"))), + Literal(fmt3), + timeZoneId), + MICROSECONDS.toSeconds(DateTimeUtils + .daysToMicros(DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), zid)) + ) + val t1 = ToUnixTimestamp(CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + val t2 = ToUnixTimestamp(CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] + assert(t2 - t1 <= 1) + checkEvaluation( + ToUnixTimestamp( + Literal.create(null, DateType), + Literal.create(null, StringType), + timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal.create(null, DateType), Literal(fmt1), timeZoneId), + null) + checkEvaluation( + ToUnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), + MICROSECONDS.toSeconds( + DateTimeUtils.daysToMicros(DateTimeUtils.fromJavaDate(date1), zid)) + ) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = ToUnixTimestamp( + BoundReference(ordinal = 0, dataType = StringType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = 0L, + inputRow = InternalRow( + UTF8String.fromString(sdf1.format(new Timestamp(0))), + UTF8String.fromString(fmt1)) + ) + } + } + } + } + // Test escaping of format + GenerateUnsafeProjection.generate( + ToUnixTimestamp(Literal("2015-07-24"), Literal("\""), UTC_OPT) :: Nil) + } + + // Modified based on vanilla spark to explicitly set timezone in config. + test(GLUTEN_TEST + "DateFormat") { + val PST_OPT = Option(PST.getId) + val JST_OPT = Option(JST.getId) + + Seq("legacy", "corrected").foreach { + legacyParserPolicy => + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> UTC_OPT.get) { + checkEvaluation( + DateFormatClass(Literal.create(null, TimestampType), Literal("y"), UTC_OPT), + null) + checkEvaluation( + DateFormatClass( + Cast(Literal(d), TimestampType, UTC_OPT), + Literal.create(null, StringType), + UTC_OPT), + null) + + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("y"), UTC_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), UTC_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, UTC_OPT), Literal("H"), UTC_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), UTC_OPT), "13") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("y"), PST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), PST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, PST_OPT), Literal("H"), PST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), PST_OPT), "5") + } + + withSQLConf( + SQLConf.LEGACY_TIME_PARSER_POLICY.key -> legacyParserPolicy, + SQLConf.SESSION_LOCAL_TIMEZONE.key -> PST_OPT.get) { + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("y"), JST_OPT), + "2015") + checkEvaluation(DateFormatClass(Literal(ts), Literal("y"), JST_OPT), "2013") + checkEvaluation( + DateFormatClass(Cast(Literal(d), TimestampType, JST_OPT), Literal("H"), JST_OPT), + "0") + checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), JST_OPT), "22") + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala new file mode 100644 index 000000000000..8f9054928e40 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenDecimalExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenDecimalExpressionSuite extends DecimalExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala new file mode 100644 index 000000000000..4f9d1ffff271 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenHashExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenHashExpressionsSuite extends HashExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala new file mode 100644 index 000000000000..2b8aec03d7bd --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenIntervalExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenIntervalExpressionsSuite extends IntervalExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala new file mode 100644 index 000000000000..556d185af078 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenLiteralExpressionSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenLiteralExpressionSuite extends LiteralExpressionSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala new file mode 100644 index 000000000000..b37107b27e3c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMathExpressionsSuite.scala @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestConstants +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.types._ + +class GlutenMathExpressionsSuite extends MathExpressionsSuite with GlutenTestsTrait { + test(GlutenTestConstants.GLUTEN_TEST + "round/bround/floor/ceil") { + val scales = -6 to 6 + val doublePi: Double = math.Pi + val shortPi: Short = 31415 + val intPi: Int = 314159265 + val longPi: Long = 31415926535897932L + val bdPi: BigDecimal = BigDecimal(31415927L, 7) + val floatPi: Float = 3.1415f + + val doubleResults: Seq[Double] = + Seq(0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 3.0, 3.1, 3.14, 3.142, 3.1416, 3.14159, 3.141593) + + val floatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.142f, 3.1415f, 3.1415f, 3.1415f) + + val bRoundFloatResults: Seq[Float] = + Seq(0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 0.0f, 3.0f, 3.1f, 3.14f, 3.141f, 3.1415f, 3.1415f, 3.1415f) + + val shortResults: Seq[Short] = Seq[Short](0, 0, 30000, 31000, 31400, 31420) ++ + Seq.fill[Short](7)(31415) + + val intResults: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + + val longResults: Seq[Long] = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535897900L, 31415926535897930L) ++ + Seq.fill(7)(31415926535897932L) + + val intResultsB: Seq[Int] = + Seq(314000000, 314200000, 314160000, 314159000, 314159300, 314159260) ++ Seq.fill(7)( + 314159265) + + def doubleResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.14159, 3.141592) + Decimal(results(i)) + } + + def doubleResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1416, 3.1416, 3.141593) + Decimal(results(i)) + } + + def floatResultsFloor(i: Int): Any = { + val results = Seq(0, 0, 0, 0, 0, 0, 3, 3.1, 3.14, 3.141, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def floatResultsCeil(i: Int): Any = { + val results = + Seq(1000000, 100000, 10000, 1000, 100, 10, 4, 3.2, 3.15, 3.142, 3.1415, 3.1415, 3.1415) + Decimal(results(i)) + } + + def shortResultsFloor(i: Int): Decimal = { + val results = Seq(0, 0, 30000, 31000, 31400, 31410) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def shortResultsCeil(i: Int): Decimal = { + val results = Seq(1000000, 100000, 40000, 32000, 31500, 31420) ++ Seq.fill(7)(31415) + Decimal(results(i)) + } + + def longResultsFloor(i: Int): Decimal = { + val results = Seq(31415926535000000L, 31415926535800000L, 31415926535890000L, + 31415926535897000L, 31415926535897900L, 31415926535897930L, 31415926535897932L) ++ + Seq.fill(6)(31415926535897932L) + Decimal(results(i)) + } + + def longResultsCeil(i: Int): Decimal = { + val results = Seq(31415926536000000L, 31415926535900000L, 31415926535900000L, + 31415926535898000L, 31415926535898000L, 31415926535897940L) ++ + Seq.fill(7)(31415926535897932L) + Decimal(results(i)) + } + + def intResultsFloor(i: Int): Decimal = { + val results = + Seq(314000000, 314100000, 314150000, 314159000, 314159200, 314159260) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + def intResultsCeil(i: Int): Decimal = { + val results = + Seq(315000000, 314200000, 314160000, 314160000, 314159300, 314159270) ++ Seq.fill(7)( + 314159265) + Decimal(results(i)) + } + + scales.zipWithIndex.foreach { + case (scale, i) => + checkEvaluation(Round(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(Round(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(Round(intPi, scale), intResults(i), EmptyRow) + checkEvaluation(Round(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(Round(floatPi, scale), floatResults(i), EmptyRow) + checkEvaluation(BRound(doublePi, scale), doubleResults(i), EmptyRow) + checkEvaluation(BRound(shortPi, scale), shortResults(i), EmptyRow) + checkEvaluation(BRound(intPi, scale), intResultsB(i), EmptyRow) + checkEvaluation(BRound(longPi, scale), longResults(i), EmptyRow) + checkEvaluation(BRound(floatPi, scale), bRoundFloatResults(i), EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(doublePi), Literal(scale))), + doubleResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(shortPi), Literal(scale))), + shortResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(intPi), Literal(scale))), + intResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(longPi), Literal(scale))), + longResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal(floatPi), Literal(scale))), + floatResultsFloor(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(doublePi), Literal(scale))), + doubleResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(shortPi), Literal(scale))), + shortResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(intPi), Literal(scale))), + intResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(longPi), Literal(scale))), + longResultsCeil(i), + EmptyRow) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal(floatPi), Literal(scale))), + floatResultsCeil(i), + EmptyRow) + } + + val bdResults: Seq[BigDecimal] = Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14159"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + val bdResultsFloor: Seq[BigDecimal] = + Seq( + BigDecimal(3), + BigDecimal("3.1"), + BigDecimal("3.14"), + BigDecimal("3.141"), + BigDecimal("3.1415"), + BigDecimal("3.14159"), + BigDecimal("3.141592"), + BigDecimal("3.1415927") + ) + + val bdResultsCeil: Seq[BigDecimal] = Seq( + BigDecimal(4), + BigDecimal("3.2"), + BigDecimal("3.15"), + BigDecimal("3.142"), + BigDecimal("3.1416"), + BigDecimal("3.14160"), + BigDecimal("3.141593"), + BigDecimal("3.1415927") + ) + + (0 to 7).foreach { + i => + checkEvaluation(Round(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(BRound(bdPi, i), bdResults(i), EmptyRow) + checkEvaluation(RoundFloor(bdPi, i), bdResultsFloor(i), EmptyRow) + checkEvaluation(RoundCeil(bdPi, i), bdResultsCeil(i), EmptyRow) + } + (8 to 10).foreach { + scale => + checkEvaluation(Round(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(BRound(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundFloor(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(RoundCeil(bdPi, scale), bdPi, EmptyRow) + } + + DataTypeTestUtils.numericTypes.foreach { + dataType => + checkEvaluation(Round(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + Round(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation(BRound(Literal.create(null, dataType), Literal(2)), null) + checkEvaluation( + BRound(Literal.create(null, dataType), Literal.create(null, IntegerType)), + null) + checkEvaluation( + checkDataTypeAndCast(RoundFloor(Literal.create(null, dataType), Literal(2))), + null) + checkEvaluation( + checkDataTypeAndCast(RoundCeil(Literal.create(null, dataType), Literal(2))), + null) + } + + checkEvaluation(Round(2.5, 0), 3.0) + checkEvaluation(Round(3.5, 0), 4.0) + checkEvaluation(Round(-2.5, 0), -3.0) + checkEvaluation(Round(-3.5, 0), -4.0) + checkEvaluation(Round(-0.35, 1), -0.4) + checkEvaluation(Round(-35, -1), -40) + checkEvaluation(Round(BigDecimal("45.00"), -1), BigDecimal(50)) + checkEvaluation(BRound(2.5, 0), 2.0) + checkEvaluation(BRound(3.5, 0), 4.0) + checkEvaluation(BRound(-2.5, 0), -2.0) + checkEvaluation(BRound(-3.5, 0), -4.0) + checkEvaluation(BRound(-0.35, 1), -0.4) + checkEvaluation(BRound(-35, -1), -40) + checkEvaluation(BRound(BigDecimal("45.00"), -1), BigDecimal(40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(2.5), Literal(0))), Decimal(2)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-2.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-3.5), Literal(0))), Decimal(-4L)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.35), Literal(1))), Decimal(-0.4)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-35), Literal(-1))), Decimal(-40)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(-0.1), Literal(0))), Decimal(-1)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(3.1411), Literal(-3))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundFloor(Literal(135.135), Literal(-2))), Decimal(100)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(2.5), Literal(0))), Decimal(3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.5), Literal(0))), Decimal(4L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-2.5), Literal(0))), Decimal(-2L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-3.5), Literal(0))), Decimal(-3L)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.35), Literal(1))), Decimal(-0.3)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-35), Literal(-1))), Decimal(-30)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(-0.1), Literal(0))), Decimal(0)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(5), Literal(0))), Decimal(5)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(3.1411), Literal(-3))), Decimal(1000)) + checkEvaluation(checkDataTypeAndCast(RoundCeil(Literal(135.135), Literal(-2))), Decimal(200)) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala new file mode 100644 index 000000000000..c734a9cfbbdc --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenMiscExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenMiscExpressionsSuite extends MiscExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala new file mode 100644 index 000000000000..34830b368cae --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNondeterministicSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNondeterministicSuite extends NondeterministicSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala new file mode 100644 index 000000000000..900fd764d0d9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenNullExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenNullExpressionsSuite extends NullExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala new file mode 100644 index 000000000000..90e93f3593ee --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenPredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenPredicateSuite extends PredicateSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala new file mode 100644 index 000000000000..95d2e71ffe59 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRandomSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRandomSuite extends RandomSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala new file mode 100644 index 000000000000..33cb9a783585 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenRegexpExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenRegexpExpressionsSuite extends RegexpExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala new file mode 100644 index 000000000000..37c630f495f2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenSortOrderExpressionsSuite.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestsTrait + +class GlutenSortOrderExpressionsSuite extends SortOrderExpressionsSuite with GlutenTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.scala new file mode 100644 index 000000000000..aa2beef50f54 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/catalyst/expressions/GlutenStringExpressionsSuite.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.spark.sql.catalyst.expressions + +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.GlutenTestsTrait +import org.apache.spark.sql.types._ + +class GlutenStringExpressionsSuite extends StringExpressionsSuite with GlutenTestsTrait { + test(GLUTEN_TEST + "concat") { + def testConcat(inputs: String*): Unit = { + val expected = if (inputs.contains(null)) null else inputs.mkString + checkEvaluation(Concat(inputs.map(Literal.create(_, StringType))), expected) + } + + // testConcat() velox not supported + testConcat(null) + testConcat("") + testConcat("ab") + testConcat("a", "b") + testConcat("a", "b", "C") + testConcat("a", null, "C") + testConcat("a", null, null) + testConcat(null, null, null) + + // scalastyle:off + // non ascii characters are not allowed in the code, so we disable the scalastyle here. + testConcat("数据", null, "砖头") + // scalastyle:on + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala new file mode 100644 index 000000000000..4099ea138227 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSessionCatalogSuite + extends DataSourceV2DataFrameSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala new file mode 100644 index 000000000000..327c930bfb3f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2DataFrameSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2DataFrameSuite + extends DataSourceV2DataFrameSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala new file mode 100644 index 000000000000..10f4d90f54f5 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2FunctionSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2FunctionSuite + extends DataSourceV2FunctionSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala new file mode 100644 index 000000000000..7e1a1cdaca9a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSessionCatalogSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2SQLSessionCatalogSuite + extends DataSourceV2SQLSessionCatalogSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala new file mode 100644 index 000000000000..ff7618008680 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV1Filter.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql._ + +class GlutenDataSourceV2SQLSuiteV1Filter + extends DataSourceV2SQLSuiteV1Filter + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala new file mode 100644 index 000000000000..7e02fc07cec0 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2SQLSuiteV2Filter.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql._ + +class GlutenDataSourceV2SQLSuiteV2Filter + extends DataSourceV2SQLSuiteV2Filter + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.scala new file mode 100644 index 000000000000..4ca40a1093c4 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDataSourceV2Suite.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.spark.sql.connector + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.execution.ColumnarShuffleExchangeExec +import org.apache.spark.sql.internal.SQLConf + +import test.org.apache.spark.sql.connector.JavaPartitionAwareDataSource + +class GlutenDataSourceV2Suite extends DataSourceV2Suite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + test("Gluten: partitioning reporting") { + import org.apache.spark.sql.functions.{count, sum} + withSQLConf(SQLConf.V2_BUCKETING_ENABLED.key -> "true") { + Seq(classOf[PartitionAwareDataSource], classOf[JavaPartitionAwareDataSource]).foreach { + cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + checkAnswer(df, Seq(Row(1, 4), Row(1, 4), Row(3, 6), Row(2, 6), Row(4, 2), Row(4, 2))) + + val groupByColA = df.groupBy('i).agg(sum('j)) + checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4))) + assert(collectFirst(groupByColA.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColAB = df.groupBy('i, 'j).agg(count("*")) + checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2))) + assert(collectFirst(groupByColAB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isEmpty) + + val groupByColB = df.groupBy('j).agg(sum('i)) + checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) + assert(collectFirst(groupByColB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + + val groupByAPlusB = df.groupBy('i + 'j).agg(count("*")) + checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) + assert(collectFirst(groupByAPlusB.queryExecution.executedPlan) { + case e: ColumnarShuffleExchangeExec => e + }.isDefined) + } + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala new file mode 100644 index 000000000000..ea2fc4e943e1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenDeleteFromTableSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDeleteFromTableSuite + extends GroupBasedDeleteFromTableSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala new file mode 100644 index 000000000000..99570522cf31 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenFileDataSourceV2FallBackSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileDataSourceV2FallBackSuite + extends FileDataSourceV2FallBackSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.scala new file mode 100644 index 000000000000..2d2d5b3198c6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenKeyGroupedPartitioningSuite.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.connector + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenKeyGroupedPartitioningSuite + extends KeyGroupedPartitioningSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala new file mode 100644 index 000000000000..735b5d1a0e1b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenLocalScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenLocalScanSuite extends LocalScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala new file mode 100644 index 000000000000..59a14fb11c00 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenMetadataColumnSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenMetadataColumnSuite extends MetadataColumnSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala new file mode 100644 index 000000000000..92f2a04cebe1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenSupportsCatalogOptionsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSupportsCatalogOptionsSuite + extends SupportsCatalogOptionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala new file mode 100644 index 000000000000..93502b7adb05 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenTableCapabilityCheckSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.connector + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTableCapabilityCheckSuite + extends TableCapabilityCheckSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.scala new file mode 100644 index 000000000000..f96ec9a6d1df --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/connector/GlutenWriteDistributionAndOrderingSuite.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.connector + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenWriteDistributionAndOrderingSuite + extends WriteDistributionAndOrderingSuite + with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = { + // Native SQL configs + super.sparkConf + .set("spark.sql.shuffle.partitions", "5") + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala new file mode 100644 index 000000000000..6c14c16664a7 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsDSv2Suite.scala @@ -0,0 +1,23 @@ +/* + * 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.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsDSv2Suite + extends QueryCompilationErrorsDSv2Suite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala new file mode 100644 index 000000000000..7ccb3b059ac6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryCompilationErrorsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryCompilationErrorsSuite + extends QueryCompilationErrorsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.scala new file mode 100644 index 000000000000..8896541c29d2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryExecutionErrorsSuite.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.sql.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryExecutionErrorsSuite + extends QueryExecutionErrorsSuite + with GlutenSQLTestsBaseTrait { + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala new file mode 100644 index 000000000000..307a740396ea --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/errors/GlutenQueryParsingErrorsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.errors + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenQueryParsingErrorsSuite extends QueryParsingErrorsSuite with GlutenSQLTestsBaseTrait {} 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 new file mode 100644 index 000000000000..6434e0040f1d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/FallbackStrategiesSuite.scala @@ -0,0 +1,204 @@ +/* + * 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 io.glutenproject.backendsapi.BackendsApiManager +import io.glutenproject.execution.BasicScanExecTransformer +import io.glutenproject.extension.{ColumnarOverrideRules, GlutenPlan, InsertTransitions} +import io.glutenproject.extension.columnar.{TRANSFORM_UNSUPPORTED, TransformHints} +import io.glutenproject.utils.QueryPlanSelector + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute + +class FallbackStrategiesSuite extends GlutenSQLTestsTrait { + + test("Fall back the whole query if one unsupported") { + withSQLConf(("spark.gluten.sql.columnar.query.fallback.threshold", "1")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = ColumnarOverrideRules(spark) + rule.preColumnarTransitions(originalPlan) + // Fake output of preColumnarTransitions, mocking replacing UnaryOp1 with UnaryOp1Transformer. + val planAfterPreOverride = + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + val planWithTransition = InsertTransitions.insertTransitions(planAfterPreOverride, false) + val outputPlan = rule.postColumnarTransitions(planWithTransition) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + + test("Fall back the whole plan if meeting the configured threshold") { + withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "1")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = ColumnarOverrideRules(spark) + rule.preColumnarTransitions(originalPlan) + rule.enableAdaptiveContext() + // Fake output of preColumnarTransitions, mocking replacing UnaryOp1 with UnaryOp1Transformer. + val planAfterPreOverride = + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + val planWithTransition = InsertTransitions.insertTransitions(planAfterPreOverride, false) + val outputPlan = rule.postColumnarTransitions(planWithTransition) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + + test("Don't fall back the whole plan if NOT meeting the configured threshold") { + withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "4")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = ColumnarOverrideRules(spark) + rule.preColumnarTransitions(originalPlan) + rule.enableAdaptiveContext() + // Fake output of preColumnarTransitions, mocking replacing UnaryOp1 with UnaryOp1Transformer. + val planAfterPreOverride = + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOp())))) + val planWithTransition = InsertTransitions.insertTransitions(planAfterPreOverride, false) + val outputPlan = rule.postColumnarTransitions(planWithTransition) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + + test( + "Fall back the whole plan if meeting the configured threshold (leaf node is" + + " transformable)") { + withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "2")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = ColumnarOverrideRules(spark) + rule.preColumnarTransitions(originalPlan) + rule.enableAdaptiveContext() + // Fake output of preColumnarTransitions, mocking replacing UnaryOp1 with UnaryOp1Transformer + // and replacing LeafOp with LeafOpTransformer. + val planAfterPreOverride = + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + val planWithTransition = InsertTransitions.insertTransitions(planAfterPreOverride, false) + val outputPlan = rule.postColumnarTransitions(planWithTransition) + // Expect to fall back the entire plan. + assert(outputPlan == originalPlan) + } + } + + test( + "Don't Fall back the whole plan if NOT meeting the configured threshold (" + + "leaf node is transformable)") { + withSQLConf(("spark.gluten.sql.columnar.wholeStage.fallback.threshold", "3")) { + val originalPlan = UnaryOp2(UnaryOp1(UnaryOp2(UnaryOp1(LeafOp())))) + val rule = ColumnarOverrideRules(spark) + rule.preColumnarTransitions(originalPlan) + rule.enableAdaptiveContext() + // Fake output of preColumnarTransitions, mocking replacing UnaryOp1 with UnaryOp1Transformer + // and replacing LeafOp with LeafOpTransformer. + val planAfterPreOverride = + UnaryOp2(UnaryOp1Transformer(UnaryOp2(UnaryOp1Transformer(LeafOpTransformer())))) + val planWithTransition = InsertTransitions.insertTransitions(planAfterPreOverride, false) + val outputPlan = rule.postColumnarTransitions(planWithTransition) + // Expect to get the plan with columnar rule applied. + assert(outputPlan != originalPlan) + } + } + + test("Tag not transformable more than once") { + val originalPlan = UnaryOp1(LeafOp(supportsColumnar = true)) + TransformHints.tag(originalPlan, TRANSFORM_UNSUPPORTED(Some("fake reason"))) + val rule = ColumnarOverrideRules(spark) + val newPlan = rule.preColumnarTransitions(originalPlan) + val reason = TransformHints.getHint(newPlan).asInstanceOf[TRANSFORM_UNSUPPORTED].reason + assert(reason.isDefined) + if (BackendsApiManager.getSettings.fallbackOnEmptySchema(newPlan)) { + assert( + reason.get.contains("fake reason") && + reason.get.contains("at least one of its children has empty output")) + } else { + assert(reason.get.contains("fake reason")) + } + } + + test("test enabling/disabling Gluten at thread level") { + spark.sql("create table fallback_by_thread_config (a int) using parquet") + spark.sql("insert overwrite fallback_by_thread_config select id as a from range(3)") + val sql = + """ + |select * + |from fallback_by_thread_config as t0 + |""".stripMargin + + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + + val thread = new Thread( + () => { + spark.sparkContext.setLocalProperty(QueryPlanSelector.GLUTEN_ENABLE_FOR_THREAD_KEY, "false") + val fallbackPlan = spark.sql(sql).queryExecution.executedPlan + val fallbackScanExec = fallbackPlan.collect { + case e: FileSourceScanExec if !e.isInstanceOf[BasicScanExecTransformer] => true + } + assert(fallbackScanExec.size == 1) + + spark.sparkContext.setLocalProperty(QueryPlanSelector.GLUTEN_ENABLE_FOR_THREAD_KEY, null) + val noFallbackPlan = spark.sql(sql).queryExecution.executedPlan + val noFallbackScanExec = noFallbackPlan.collect { case _: BasicScanExecTransformer => true } + assert(noFallbackScanExec.size == 1) + }) + thread.start() + thread.join(10000) + } +} + +case class LeafOp(override val supportsColumnar: Boolean = false) extends LeafExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = Seq.empty +} + +case class UnaryOp1(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1 = + copy(child = newChild) +} + +case class UnaryOp2(child: SparkPlan, override val supportsColumnar: Boolean = false) + extends UnaryExecNode { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp2 = + copy(child = newChild) +} + +// For replacing LeafOp. +case class LeafOpTransformer(override val supportsColumnar: Boolean = true) + extends LeafExecNode + with GlutenPlan { + 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) + extends UnaryExecNode + with GlutenPlan { + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException() + override def output: Seq[Attribute] = child.output + override protected def withNewChildInternal(newChild: SparkPlan): UnaryOp1Transformer = + copy(child = newChild) +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala new file mode 100644 index 000000000000..481863354227 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenBroadcastExchangeSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBroadcastExchangeSuite extends BroadcastExchangeSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala new file mode 100644 index 000000000000..bc80c1b8f3e3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenCoalesceShufflePartitionsSuite.scala @@ -0,0 +1,295 @@ +/* + * 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.spark.SparkConf +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{GlutenTestsCommonTrait, QueryTest, SparkSession} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf + +class GlutenCoalesceShufflePartitionsSuite + extends CoalesceShufflePartitionsSuite + with GlutenTestsCommonTrait { + + override protected def afterAll(): Unit = {} + + override def withSparkSession( + f: SparkSession => Unit, + targetPostShuffleInputSize: Int, + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { + val sparkConf = + new SparkConf(false) + .setMaster("local[*]") + .setAppName("test") + .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") + .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") + .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") + .set(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, targetPostShuffleInputSize.toString) + .set(SQLConf.COALESCE_PARTITIONS_ENABLED.key, "true") + // Gluten config + .set("spark.plugins", "io.glutenproject.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "5g") + minNumPostShufflePartitions match { + case Some(numPartitions) => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, numPartitions.toString) + case None => + sparkConf.set(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key, "1") + } + + val spark = SparkSession + .builder() + .config(sparkConf) + .getOrCreate() + try f(spark) + finally { + spark.stop() + } + } + + Seq(Some(5), None).foreach { + minNumPostShufflePartitions => + val testNameNote = minNumPostShufflePartitions match { + case Some(numPartitions) => "(minNumPostShufflePartitions: " + numPartitions + ")" + case None => "" + } + + // Ported from vanilla spark with targetPostShuffleInputSize changed. + test(GLUTEN_TEST + s"determining the number of reducers: aggregate operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 20 as key", "id as value") + val agg = df.groupBy("key").count() + + // Check the answer first. + QueryTest.checkAnswer(agg, spark.range(0, 20).selectExpr("id", "50 as cnt").collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = agg.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + case None => + assert(shuffleReads.length === 1) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + // Change the original value 2000 to 6000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 6000, minNumPostShufflePartitions) + } + + test(GLUTEN_TEST + s"determining the number of reducers: join operator$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "id as value") + .union(spark.range(0, 1000).selectExpr("id % 500 as key", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + // Change the original value 16384 to 40000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 40000, minNumPostShufflePartitions) + } + + test(GLUTEN_TEST + s"determining the number of reducers: complex query 1$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + .groupBy("key2") + .count() + .toDF("key2", "cnt2") + + val join = df1.join(df2, col("key1") === col("key2")).select(col("key1"), col("cnt2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 500) + .selectExpr("id", "2 as cnt") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 2)) + } + } + + // Change the original value 16384 to 40000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 40000, minNumPostShufflePartitions) + } + + test(GLUTEN_TEST + s"determining the number of reducers: complex query 2$testNameNote") { + val test: (SparkSession) => Unit = { + spark: SparkSession => + val df1 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key1", "id as value1") + .groupBy("key1") + .count() + .toDF("key1", "cnt1") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = + df1 + .join(df2, col("key1") === col("key2")) + .select(col("key1"), col("cnt1"), col("value2")) + + // Check the answer first. + val expectedAnswer = + spark + .range(0, 1000) + .selectExpr("id % 500 as key", "2 as cnt", "id as value") + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's look at the number of post-shuffle partitions estimated + // by the ExchangeCoordinator. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + + minNumPostShufflePartitions match { + case Some(numPartitions) => + assert(shuffleReads.isEmpty) + + case None => + assert(shuffleReads.length === 2) + shuffleReads.foreach(read => assert(read.outputPartitioning.numPartitions === 3)) + } + } + + // Change the original value 12000 to 30000 for gluten. The test depends on the calculation + // for bytesByPartitionId in MapOutputStatistics. Gluten has a different statistic result. + // See ShufflePartitionsUtil.coalescePartitions & GlutenColumnarShuffleWriter's mapStatus. + withSparkSession(test, 30000, minNumPostShufflePartitions) + } + + test( + GLUTEN_TEST + s"determining the number of reducers:" + + s" plan already partitioned$testNameNote") { + val test: SparkSession => Unit = { + spark: SparkSession => + try { + spark.range(1000).write.bucketBy(30, "id").saveAsTable("t") + // `df1` is hash partitioned by `id`. + val df1 = spark.read.table("t") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("id") === col("key2")).select(col("id"), col("value2")) + + // Check the answer first. + val expectedAnswer = spark + .range(0, 500) + .selectExpr("id % 500", "id as value") + .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) + QueryTest.checkAnswer(join, expectedAnswer.collect()) + + // Then, let's make sure we do not reduce number of post shuffle partitions. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .executedPlan + val shuffleReads = finalPlan.collect { case r @ CoalescedShuffleRead() => r } + assert(shuffleReads.length === 0) + } finally { + spark.sql("drop table t") + } + } + withSparkSession(test, 12000, minNumPostShufflePartitions) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala new file mode 100644 index 000000000000..ea2670264e8c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenExchangeSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.exchange.{Exchange, ReusedExchangeExec} +import org.apache.spark.sql.internal.SQLConf + +class GlutenExchangeSuite extends ExchangeSuite with GlutenSQLTestsBaseTrait { + + test("Exchange reuse across the whole plan with shuffle partition 2") { + // The shuffle exchange will be inserted between Aggregate + // when shuffle partition is > 1. + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", + SQLConf.SHUFFLE_PARTITIONS.key -> "2") { + val df = sql(""" + |SELECT + | (SELECT max(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key), + | a.key + |FROM testData AS a + |JOIN testData AS b ON b.key = a.key + """.stripMargin) + + val plan = df.queryExecution.executedPlan + + val exchangeIds = plan.collectWithSubqueries { case e: Exchange => e.id } + val reusedExchangeIds = plan.collectWithSubqueries { + case re: ReusedExchangeExec => re.child.id + } + + assert(exchangeIds.size == 2, "Whole plan exchange reusing not working correctly") + assert(reusedExchangeIds.size == 3, "Whole plan exchange reusing not working correctly") + assert( + reusedExchangeIds.forall(exchangeIds.contains(_)), + "ReusedExchangeExec should reuse an existing exchange") + + val df2 = sql(""" + |SELECT + | (SELECT min(a.key) FROM testData AS a JOIN testData AS b ON b.key = a.key), + | (SELECT max(a.key) FROM testData AS a JOIN testData2 AS b ON b.a = a.key) + """.stripMargin) + + val plan2 = df2.queryExecution.executedPlan + + val exchangeIds2 = plan2.collectWithSubqueries { case e: Exchange => e.id } + val reusedExchangeIds2 = plan2.collectWithSubqueries { + case re: ReusedExchangeExec => re.child.id + } + + assert(exchangeIds2.size == 4, "Whole plan exchange reusing not working correctly") + assert(reusedExchangeIds2.size == 2, "Whole plan exchange reusing not working correctly") + assert( + reusedExchangeIds2.forall(exchangeIds2.contains(_)), + "ReusedExchangeExec should reuse an existing exchange") + } + } +} 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 new file mode 100644 index 000000000000..94c83a3c490e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReplaceHashWithSortAggSuite.scala @@ -0,0 +1,146 @@ +/* + * 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 io.glutenproject.execution.HashAggregateExecBaseTransformer + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.execution.aggregate.{ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.internal.SQLConf + +class GlutenReplaceHashWithSortAggSuite + extends ReplaceHashWithSortAggSuite + with GlutenSQLTestsBaseTrait { + + private def checkNumAggs(df: DataFrame, hashAggCount: Int, sortAggCount: Int): Unit = { + val plan = df.queryExecution.executedPlan + assert(collectWithSubqueries(plan) { + case s @ (_: HashAggregateExecBaseTransformer | _: ObjectHashAggregateExec) => s + }.length == hashAggCount) + assert(collectWithSubqueries(plan) { case s: SortAggregateExec => s }.length == sortAggCount) + } + + private def checkAggs( + query: String, + enabledHashAggCount: Int, + enabledSortAggCount: Int, + disabledHashAggCount: Int, + disabledSortAggCount: Int): Unit = { + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "true") { + val df = sql(query) + checkNumAggs(df, enabledHashAggCount, enabledSortAggCount) + val result = df.collect() + withSQLConf(SQLConf.REPLACE_HASH_WITH_SORT_AGG_ENABLED.key -> "false") { + val df = sql(query) + checkNumAggs(df, disabledHashAggCount, disabledSortAggCount) + checkAnswer(df, result) + } + } + } + + // === Following cases override super class's cases === + + test(GLUTEN_TEST + "replace partial hash aggregate with sort aggregate") { + withTempView("t") { + spark.range(100).selectExpr("id as key").repartition(10).createOrReplaceTempView("t") + + Seq("FIRST", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT key + | FROM t + | WHERE key > 10 + | SORT BY key + |) + |GROUP BY key + """.stripMargin + aggExpr match { + case "FIRST" => + checkAggs(query, 2, 0, 2, 0) + case _ => + checkAggs(query, 1, 1, 2, 0) + } + } + } + } + + test(GLUTEN_TEST + "replace partial and final hash aggregate together with sort aggregate") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT /*+ SHUFFLE_MERGE(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + test(GLUTEN_TEST + "do not replace hash aggregate if child does not have sort order") { + withTempView("t1", "t2") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + spark.range(50).selectExpr("id as key").createOrReplaceTempView("t2") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT key, $aggExpr(key) + |FROM + |( + | SELECT /*+ BROADCAST(t1) */ t1.key AS key + | FROM t1 + | JOIN t2 + | ON t1.key = t2.key + |) + |GROUP BY key + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } + + test(GLUTEN_TEST + "do not replace hash aggregate if there is no group-by column") { + withTempView("t1") { + spark.range(100).selectExpr("id as key").createOrReplaceTempView("t1") + Seq("COUNT", "COLLECT_LIST").foreach { + aggExpr => + val query = + s""" + |SELECT $aggExpr(key) + |FROM t1 + """.stripMargin + checkAggs(query, 2, 0, 2, 0) + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala new file mode 100644 index 000000000000..d7232f6a06c8 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenReuseExchangeAndSubquerySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenReuseExchangeAndSubquerySuite + extends ReuseExchangeAndSubquerySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala new file mode 100644 index 000000000000..67b0eb277e5e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSQLWindowFunctionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.GlutenSQLTestsTrait + +class GlutenSQLWindowFunctionSuite extends SQLWindowFunctionSuite with GlutenSQLTestsTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala new file mode 100644 index 000000000000..de9a897ffb09 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSameResultSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSameResultSuite extends SameResultSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala new file mode 100644 index 000000000000..d43a7fea041b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenSortSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSortSuite extends SortSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala new file mode 100644 index 000000000000..bc231e52adc3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/GlutenTakeOrderedAndProjectSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenTakeOrderedAndProjectSuite + extends TakeOrderedAndProjectSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala new file mode 100644 index 000000000000..0d45df3a5ada --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/adaptive/GlutenAdaptiveQueryExecSuite.scala @@ -0,0 +1,1501 @@ +/* + * 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.adaptive + +import io.glutenproject.execution.{BroadcastHashJoinExecTransformer, ShuffledHashJoinExecTransformerBase, SortExecTransformer, SortMergeJoinExecTransformer} + +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.{Dataset, GlutenSQLTestsTrait, Row} +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange.{ENSURE_REQUIREMENTS, Exchange, REPARTITION_BY_COL, REPARTITION_BY_NUM, ReusedExchangeExec, ShuffleExchangeExec, ShuffleExchangeLike, ShuffleOrigin} +import org.apache.spark.sql.execution.joins.{BaseJoinExec, BroadcastHashJoinExec, ShuffledHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.functions.when +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SQLTestData.TestData +import org.apache.spark.sql.types.{IntegerType, StructType} + +import org.apache.logging.log4j.Level + +class GlutenAdaptiveQueryExecSuite extends AdaptiveQueryExecSuite with GlutenSQLTestsTrait { + import testImplicits._ + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.gluten.sql.columnar.forceShuffledHashJoin", "false") + .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + } + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + var finalPlanCnt = 0 + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case SparkListenerSQLAdaptiveExecutionUpdate(_, _, sparkPlanInfo) => + if (sparkPlanInfo.simpleString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) { + finalPlanCnt += 1 + } + case _ => // ignore other events + } + } + } + spark.sparkContext.addSparkListener(listener) + + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan isFinalPlan=false")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + checkAnswer(df, result) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan isFinalPlan=true")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + spark.sparkContext.listenerBus.waitUntilEmpty() + // AQE will post `SparkListenerSQLAdaptiveExecutionUpdate` twice in case of subqueries that + // exist out of query stages. + val expectedFinalPlanCnt = adaptivePlan.find(_.subqueries.nonEmpty).map(_ => 2).getOrElse(1) + assert(finalPlanCnt == expectedFinalPlanCnt) + spark.sparkContext.removeSparkListener(listener) + + val exchanges = adaptivePlan.collect { case e: Exchange => e } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def broadcastHashJoinSize(plan: SparkPlan): Int = { + findTopLevelBroadcastHashJoinTransform(plan).size + findTopLevelBroadcastHashJoin(plan).size + } + + private def findTopLevelBroadcastHashJoinTransform( + plan: SparkPlan): Seq[BroadcastHashJoinExecTransformer] = { + collect(plan) { case j: BroadcastHashJoinExecTransformer => j } + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + collect(plan) { case j: BroadcastHashJoinExec => j } + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + collect(plan) { case j: SortMergeJoinExec => j } + } + + private def findTopLevelSortMergeJoinTransform( + plan: SparkPlan): Seq[SortMergeJoinExecTransformer] = { + collect(plan) { case j: SortMergeJoinExecTransformer => j } + } + + private def sortMergeJoinSize(plan: SparkPlan): Int = { + findTopLevelSortMergeJoinTransform(plan).size + findTopLevelSortMergeJoin(plan).size + } + + private def findTopLevelShuffledHashJoin(plan: SparkPlan): Seq[ShuffledHashJoinExec] = { + collect(plan) { case j: ShuffledHashJoinExec => j } + } + + private def findTopLevelShuffledHashJoinTransform( + plan: SparkPlan): Seq[ShuffledHashJoinExecTransformerBase] = { + collect(plan) { case j: ShuffledHashJoinExecTransformerBase => j } + } + + private def findTopLevelBaseJoin(plan: SparkPlan): Seq[BaseJoinExec] = { + collect(plan) { case j: BaseJoinExec => j } + } + + private def findTopLevelSort(plan: SparkPlan): Seq[SortExec] = { + collect(plan) { case s: SortExec => s } + } + + private def findTopLevelSortTransform(plan: SparkPlan): Seq[SortExecTransformer] = { + collect(plan) { case s: SortExecTransformer => s } + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedExchangeExec] = { + collectWithSubqueries(plan) { + case ShuffleQueryStageExec(_, e: ReusedExchangeExec, _) => e + case BroadcastQueryStageExec(_, e: ReusedExchangeExec, _) => e + } + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + collectWithSubqueries(plan) { case e: ReusedSubqueryExec => e } + } + + private def checkNumLocalShuffleReads( + plan: SparkPlan, + numShufflesWithoutLocalRead: Int = 0): Unit = { + val numShuffles = collect(plan) { case s: ShuffleQueryStageExec => s }.length + + val numLocalReads = collect(plan) { + case r: AQEShuffleReadExec if r.isLocalRead => r + } + // because columnar local reads cannot execute + numLocalReads.foreach { + r => + val rdd = r.executeColumnar() + val parts = rdd.partitions + assert(parts.forall(rdd.preferredLocations(_).nonEmpty)) + } + assert(numShuffles === (numLocalReads.length + numShufflesWithoutLocalRead)) + } + + test("gluten Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300" + ) { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + test("gluten Change broadcast join to merge join") { + withTable("t1", "t2") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + sql("CREATE TABLE t1 USING PARQUET AS SELECT 1 c1") + sql("CREATE TABLE t2 USING PARQUET AS SELECT 1 c1") + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM ( + | SELECT distinct c1 from t1 + | ) tmp1 JOIN ( + | SELECT distinct c1 from t2 + | ) tmp2 ON tmp1.c1 = tmp2.c1 + |""".stripMargin) + assert(broadcastHashJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 0) + assert(findTopLevelSortMergeJoinTransform(adaptivePlan).size == 1) + } + } + } + + test("gluten Reuse the parallelism of coalesced shuffle in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + } + } + + test("gluten Reuse the default parallelism in local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + val localReads = collect(adaptivePlan) { + case read: AQEShuffleReadExec if read.isLocalRead => read + } + assert(localReads.length == 2) + val localShuffleRDD0 = localReads(0) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + val localShuffleRDD1 = localReads(1) + .executeColumnar() + .asInstanceOf[ShuffledColumnarBatchRDD] + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD0.getPartitions.length == 4) + // the final parallelism is math.max(1, numReduces / numMappers): math.max(1, 5/2) = 2 + // and the partitions length is 2 * numMappers = 4 + assert(localShuffleRDD1.getPartitions.length == 4) + } + } + + test("gluten Empty stage coalesced to 1-partition RDD") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + val df1 = spark.range(10).withColumn("a", 'id) + val df2 = spark.range(10).withColumn("b", 'id) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[SortMergeJoinExecTransformer]).isDefined) + } + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1") { + val testDf = df1 + .where('a > 10) + .join(df2.where('b > 10), Seq("id"), "left_outer") + .groupBy('a) + .count() + checkAnswer(testDf, Seq()) + val plan = testDf.queryExecution.executedPlan + assert(find(plan)(_.isInstanceOf[BroadcastHashJoinExecTransformer]).isDefined) + val coalescedReads = collect(plan) { case r: AQEShuffleReadExec => r } + assert(coalescedReads.length == 3, s"$plan") + coalescedReads.foreach(r => assert(r.isLocalRead || r.partitionSpecs.length == 1)) + } + } + } + + test("gluten Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + } + + test("gluten Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + + checkNumLocalShuffleReads(adaptivePlan) + } + } + + test("gluten multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // After applied the 'OptimizeShuffleWithLocalRead' rule, we can convert all the four + // shuffle read to local shuffle read in the bottom two 'BroadcastHashJoin'. + // For the top level 'BroadcastHashJoin', the probe side is not shuffle query stage + // and the build side shuffle query stage is also converted to local shuffle read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + test("gluten multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a where t2.n = '1' + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON t2.b = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastExchange + // +-HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + test("gluten multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = + runAdaptiveAndVerifyResult(""" + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + assert(sortMergeJoinSize(plan) == 3) + assert(broadcastHashJoinSize(adaptivePlan) == 3) + + // A possible resulting query plan: + // BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- BroadcastExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- LocalShuffleReader* + // +- ShuffleExchange + // +- BroadcastHashJoin + // +- Filter + // +- HashAggregate + // +- CoalescedShuffleReader + // +- ShuffleExchange + // +- BroadcastExchange + // +-LocalShuffleReader* + // +- ShuffleExchange + + // The shuffle added by Aggregate can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 1) + } + } + + test("gluten Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "100", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + assert(sortMergeJoinSize(plan) == 3) + // TODO: vanilla spark has 2 bhj, and 1 smj, but gluten has 3 bhj, + // make sure this will not cause performance regression and why it is bhj + assert(broadcastHashJoinSize(adaptivePlan) == 1) + // Vanilla spark still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 4) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + test("gluten Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // // Even with local shuffle read, the query stage reuse can also work. + // gluten change the smj to bhj, stage is changed, so we cannot find the stage with old + // ReuseExchange from stageCache, then the reuse is removed + // https://github.com/apache/spark/pull/24706/ + // files#diff-ec42cd27662f3f528832c298a60fffa1d341feb04aa1d8c80044b70cbe0ebbfcR224 + // maybe vanilla spark should checkReuse rile again + // val ex = findReusedExchange(adaptivePlan) + // assert(ex.size == 1) + } + } + + test("gluten Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + test("gluten Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + test("gluten Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + assert(sortMergeJoinSize(plan) == 1) + assert(broadcastHashJoinSize(adaptivePlan) == 1) + checkNumLocalShuffleReads(adaptivePlan) + // Even with local shuffle read, the query stage reuse can also work. + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.child.isInstanceOf[ColumnarBroadcastExchangeExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + // Cost is equal, not test cost is greater, need new test, but other test may contain cost change, + // so it maybe not essential + test("gluten Avoid plan change if cost is greater") {} + + test("gluten Change merge join to broadcast join without local shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.LOCAL_SHUFFLE_READER_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData t1 join testData2 t2 + |ON t1.key = t2.a join testData3 t3 on t2.a = t3.a + |where t1.value = 1 + """.stripMargin + ) + assert(sortMergeJoinSize(plan) == 2) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 2) + // There is still a SMJ, and its two shuffles can't apply local read. + checkNumLocalShuffleReads(adaptivePlan, 0) + } + } + + test( + "gluten Avoid changing merge join to broadcast join if too many empty partitions " + + "on build plan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5", + // this config will make some empty partitions + SQLConf.SHUFFLE_PARTITIONS.key -> "5" + ) { + // `testData` is small enough to be broadcast but has empty partition ratio over the config. + // because testData2 in gluten sizeInBytes(from ColumnarShuffleExchangeExec plan stats) + // is 78B sometimes, so change the threshold from 80 to 60 + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "60") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.isEmpty) + } + // It is still possible to broadcast `testData2`. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2000") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + assert(sortMergeJoinSize(plan) == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + assert(bhj.head.joinBuildSide == BuildRight) + } + } + } + + test("gluten SPARK-30524: Do not optimize skew join if introduce additional shuffle") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "100", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 3 as key1", "id as value1") + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .selectExpr("id % 1 as key2", "id as value2") + .createOrReplaceTempView("skewData2") + + def checkSkewJoin(query: String, optimizeSkewJoin: Boolean): Unit = { + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult(query) + val innerSmj = findTopLevelSortMergeJoinTransform(innerAdaptivePlan) + assert(innerSmj.size == 1 && innerSmj.head.isSkewJoin == optimizeSkewJoin) + } + + // OptimizeSkewedJoin check the map status, because the + checkSkewJoin("SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2", true) + // Additional shuffle introduced, so disable the "OptimizeSkewedJoin" optimization + checkSkewJoin( + "SELECT key1 FROM skewData1 JOIN skewData2 ON key1 = key2 GROUP BY key1", + false) + } + } + } + + test("gluten SPARK-29544: adaptive skew join with different join types") { + Seq("SHUFFLE_MERGE", "SHUFFLE_HASH").foreach { + joinHint => + def getJoinNode(plan: SparkPlan): Seq[BinaryExecNode] = if (joinHint == "SHUFFLE_MERGE") { + findTopLevelSortMergeJoinTransform(plan) + } else { + findTopLevelShuffledHashJoinTransform(plan) + } + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "800" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + + def checkSkewJoin( + joins: Seq[BinaryExecNode], + leftSkewNum: Int, + rightSkewNum: Int): Unit = { + assert(joins.size == 1) + joins.head match { + case s: SortMergeJoinExecTransformer => assert(s.isSkewJoin) + case g: ShuffledHashJoinExecTransformerBase => assert(g.isSkewJoin) + case _ => assert(false) + } + assert( + joins.head.left + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == leftSkewNum) + assert( + joins.head.right + .collect { case r: AQEShuffleReadExec => r } + .head + .partitionSpecs + .collect { case p: PartialReducerPartitionSpec => p.reducerIndex } + .distinct + .length == rightSkewNum) + } + + // skewed inner join optimization + val (_, innerAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "JOIN skewData2 ON key1 = key2") + val inner = getJoinNode(innerAdaptivePlan) + // checkSkewJoin(inner, 2, 1) + + // skewed left outer join optimization + val (_, leftAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData2) */ * FROM skewData1 " + + "LEFT OUTER JOIN skewData2 ON key1 = key2") + val leftJoin = getJoinNode(leftAdaptivePlan) + // checkSkewJoin(leftJoin, 2, 0) + + // skewed right outer join optimization + val (_, rightAdaptivePlan) = runAdaptiveAndVerifyResult( + s"SELECT /*+ $joinHint(skewData1) */ * FROM skewData1 " + + "RIGHT OUTER JOIN skewData2 ON key1 = key2") + val rightJoin = getJoinNode(rightAdaptivePlan) + // checkSkewJoin(rightJoin, 0, 1) + } + } + } + } + + test("gluten SPARK-34682: AQEShuffleReadExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + val c = read.canonicalized.asInstanceOf[AQEShuffleReadExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecuteColumnar")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + + test("gluten metrics of the shuffle read") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult("SELECT key FROM testData GROUP BY key") + val reads = collect(adaptivePlan) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(!read.isLocalRead) + assert(!read.hasSkewedPartition) + assert(read.hasCoalescedPartition) + assert( + read.metrics.keys.toSeq.sorted == Seq( + "numCoalescedPartitions", + "numPartitions", + "partitionDataSize")) + assert(read.metrics("numCoalescedPartitions").value == 1) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + assert(read.metrics("partitionDataSize").value > 0) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val join = collect(adaptivePlan) { case j: BroadcastHashJoinExecTransformer => j }.head + assert(join.joinBuildSide == BuildLeft) + + val reads = collect(join.right) { case r: AQEShuffleReadExec => r } + assert(reads.length == 1) + val read = reads.head + assert(read.isLocalRead) + assert(read.metrics.keys.toSeq == Seq("numPartitions")) + assert(read.metrics("numPartitions").value == read.partitionSpecs.length) + } + + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "100", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "800", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "1000" + ) { + withTempView("skewData1", "skewData2") { + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .when('id >= 750, 1000) + .otherwise('id) + .as("key1"), + 'id.as("value1")) + .createOrReplaceTempView("skewData1") + spark + .range(0, 1000, 1, 10) + .select( + when('id < 250, 249) + .otherwise('id) + .as("key2"), + 'id.as("value2")) + .createOrReplaceTempView("skewData2") + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT * FROM skewData1 join skewData2 ON key1 = key2") + } + } + } + } + + // because gluten use columnar format, which cannot execute to get rowIterator, then get the key + // null status + ignore("gluten SPARK-32573: Eliminate NAAJ when BuildSide is HashedRelationWithAllNullKeys") {} + + // EmptyRelation case + ignore( + "gluten SPARK-35455: Unify empty relation optimization between normal and AQE optimizer " + + "- single join") {} + + test("gluten SPARK-32753: Only copy tags to node with no tags") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("v1") { + spark.range(10).union(spark.range(10)).createOrReplaceTempView("v1") + + val (_, adaptivePlan) = + runAdaptiveAndVerifyResult("SELECT id FROM v1 GROUP BY id DISTRIBUTE BY id") + assert(collect(adaptivePlan) { case s: ColumnarShuffleExchangeExec => s }.length == 1) + } + } + } + + ignore("gluten Logging plan changes for AQE") { + val testAppender = new LogAppender("plan changes") + withLogAppender(testAppender) { + withSQLConf( + // this test default level is WARN, so we should check warn level + SQLConf.PLAN_CHANGE_LOG_LEVEL.key -> "WARN", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80" + ) { + sql( + "SELECT * FROM testData JOIN testData2 ON key = a " + + "WHERE value = (SELECT max(a) FROM testData3)").collect() + } + Seq( + "=== Result of Batch AQE Preparations ===", + "=== Result of Batch AQE Post Stage Creation ===", + "=== Result of Batch AQE Replanning ===", + "=== Result of Batch AQE Query Stage Optimization ===" + ).foreach { + expectedMsg => + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains(expectedMsg))) + } + } + } + + test("gluten SPARK-33551: Do not use AQE shuffle read for repartition") { + def hasRepartitionShuffle(plan: SparkPlan): Boolean = { + find(plan) { + case s: ShuffleExchangeLike => + s.shuffleOrigin == REPARTITION_BY_COL || s.shuffleOrigin == REPARTITION_BY_NUM + case _ => false + }.isDefined + } + + def checkBHJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + probeSideLocalRead: Boolean, + probeSideCoalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + // There should be only one shuffle that can't do local read, which is either the top shuffle + // from repartition, or BHJ probe side shuffle. + checkNumLocalShuffleReads(plan, 1) + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val bhj = findTopLevelBroadcastHashJoinTransform(plan) + assert(bhj.length == 1) + + // Build side should do local read. + val buildSide = find(bhj.head.left)(_.isInstanceOf[AQEShuffleReadExec]) + assert(buildSide.isDefined) + assert(buildSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + + val probeSide = find(bhj.head.right)(_.isInstanceOf[AQEShuffleReadExec]) + if (probeSideLocalRead || probeSideCoalescedRead) { + assert(probeSide.isDefined) + if (probeSideLocalRead) { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].isLocalRead) + } else { + assert(probeSide.get.asInstanceOf[AQEShuffleReadExec].hasCoalescedPartition) + } + } else { + assert(probeSide.isEmpty) + } + } + + def checkSMJ( + df: Dataset[Row], + optimizeOutRepartition: Boolean, + optimizeSkewJoin: Boolean, + coalescedRead: Boolean): Unit = { + df.collect() + val plan = df.queryExecution.executedPlan + assert(hasRepartitionShuffle(plan) == !optimizeOutRepartition) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.length == 1) + assert(smj.head.isSkewJoin == optimizeSkewJoin) + val aqeReads = collect(smj.head) { case c: AQEShuffleReadExec => c } + if (coalescedRead || optimizeSkewJoin) { + assert(aqeReads.length == 2) + if (coalescedRead) assert(aqeReads.forall(_.hasCoalescedPartition)) + } else { + assert(aqeReads.isEmpty) + } + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "5") { + val df = sql(""" + |SELECT * FROM ( + | SELECT * FROM testData WHERE key = 1 + |) + |RIGHT OUTER JOIN testData2 + |ON value = b + """.stripMargin) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + // Repartition with no partition num specified. + checkBHJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = true + ) + + // Repartition with default partition num (5 in test env) specified. + checkBHJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out + // The final plan must have 5 partitions, no optimization can be made to the probe side. + optimizeOutRepartition = true, + probeSideLocalRead = false, + probeSideCoalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkBHJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + + // Repartition by col and project away the partition cols + checkBHJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out + optimizeOutRepartition = false, + probeSideLocalRead = true, + probeSideCoalescedRead = true + ) + } + + // Force skew join + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SKEW_JOIN_ENABLED.key -> "true", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_THRESHOLD.key -> "1", + SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR.key -> "0", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10" + ) { + // Repartition with no partition num specified. + checkSMJ( + df.repartition('b), + // The top shuffle from repartition is optimized out. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = true) + + // Repartition with default partition num (5 in test env) specified. + checkSMJ( + df.repartition(5, 'b), + // The top shuffle from repartition is optimized out. + // The final plan must have 5 partitions, can't do coalesced read. + optimizeOutRepartition = true, + optimizeSkewJoin = false, + coalescedRead = false + ) + + // Repartition with non-default partition num specified. + checkSMJ( + df.repartition(4, 'b), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false) + + // Repartition by col and project away the partition cols + checkSMJ( + df.repartition('b).select('key), + // The top shuffle from repartition is not optimized out. + optimizeOutRepartition = false, + optimizeSkewJoin = true, + coalescedRead = false + ) + } + } + } + + test("gluten SPARK-34091: Batch shuffle fetch in AQE partition coalescing") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.SHUFFLE_PARTITIONS.key -> "10", + SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "true") { + withTable("t1") { + spark.range(100).selectExpr("id + 1 as a").write.format("parquet").saveAsTable("t1") + val query = "SELECT SUM(a) FROM t1 GROUP BY a" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val metricName = SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED + val blocksFetchedMetric = collectFirst(adaptivePlan) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric.isDefined) + val blocksFetched = blocksFetchedMetric.get.value + withSQLConf(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key -> "false") { + val (_, adaptivePlan2) = runAdaptiveAndVerifyResult(query) + val blocksFetchedMetric2 = collectFirst(adaptivePlan2) { + case p if p.metrics.contains(metricName) => p.metrics(metricName) + } + assert(blocksFetchedMetric2.isDefined) + val blocksFetched2 = blocksFetchedMetric2.get.value + assert(blocksFetched == blocksFetched2) + } + } + } + } + + test("gluten SPARK-34899: Use origin plan if we can not coalesce shuffle partition") { + def checkNoCoalescePartitions(ds: Dataset[Row], origin: ShuffleOrigin): Unit = { + assert(collect(ds.queryExecution.executedPlan) { + case s: ShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => s + }.size == 1) + ds.collect() + val plan = ds.queryExecution.executedPlan + assert(collect(plan) { + case s: ColumnarShuffleExchangeExec if s.shuffleOrigin == origin && s.numPartitions == 2 => + s + }.size == 1) + checkAnswer(ds, testData) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + // Pick a small value so that no coalesce can happen. + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "100", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2" + ) { + val df = + spark.sparkContext.parallelize((1 to 100).map(i => TestData(i, i.toString)), 10).toDF() + + // partition size [1420, 1420] + checkNoCoalescePartitions(df.repartition($"key"), REPARTITION_BY_COL) + // partition size [1140, 1119] + checkNoCoalescePartitions(df.sort($"key"), ENSURE_REQUIREMENTS) + } + } + + test("gluten SPARK-35239: Coalesce shuffle partition should handle empty input RDD") { + withTable("t") { + withSQLConf( + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + spark.sql("CREATE TABLE t (c1 int) USING PARQUET") + val (_, adaptive) = runAdaptiveAndVerifyResult("SELECT c1, count(*) FROM t GROUP BY c1") + assert( + collect(adaptive) { + case c @ AQEShuffleReadExec(_, partitionSpecs) if partitionSpecs.length == 1 => + assert(c.hasCoalescedPartition) + c + }.length == 1 + ) + } + } + } + + test("gluten SPARK-35264: Support AQE side broadcastJoin threshold") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldBroadcast: Boolean): Unit = { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val (origin, adaptive) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin).size == 1) + if (shouldBroadcast) { + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive).size == 1) + } + } + } + + // t1: 1600 bytes + // t2: 160 bytes + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + checkJoinStrategy(false) + } + + withSQLConf(SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "400") { + checkJoinStrategy(true) + } + } + } + + // table partition size is different with spark + test("gluten SPARK-35264: Support AQE side shuffled hash join formula") { + withTempView("t1", "t2") { + def checkJoinStrategy(shouldShuffleHashJoin: Boolean): Unit = { + Seq("100", "100000").foreach { + size => + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> size) { + val (origin1, adaptive1) = + runAdaptiveAndVerifyResult("SELECT t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin1).size === 1) + if (shouldShuffleHashJoin && size.toInt < 100000) { + val shj = findTopLevelShuffledHashJoinTransform(adaptive1) + assert(shj.size === 1) + assert(shj.head.joinBuildSide == BuildRight) + } else { + assert(findTopLevelSortMergeJoinTransform(adaptive1).size === 1) + } + } + } + // respect user specified join hint + val (origin2, adaptive2) = runAdaptiveAndVerifyResult( + "SELECT /*+ MERGE(t1) */ t1.c1, t2.c1 FROM t1 JOIN t2 ON t1.c1 = t2.c1") + assert(findTopLevelSortMergeJoin(origin2).size === 1) + assert(findTopLevelSortMergeJoinTransform(adaptive2).size === 1) + } + + spark.sparkContext + .parallelize((1 to 100).map(i => TestData(i, i.toString)), 10) + .toDF("c1", "c2") + .createOrReplaceTempView("t1") + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 5) + .toDF("c1", "c2") + .createOrReplaceTempView("t2") + + // t1 partition size: [926, 729, 731] + // t2 partition size: [318, 120, 0] + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "3", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.PREFER_SORTMERGEJOIN.key -> "true") { + // check default value + checkJoinStrategy(false) + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "400") { + checkJoinStrategy(false) + } + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "300") { + checkJoinStrategy(false) + } + withSQLConf(SQLConf.ADAPTIVE_MAX_SHUFFLE_HASH_JOIN_LOCAL_MAP_THRESHOLD.key -> "1000") { + checkJoinStrategy(false) + } + } + } + } + + test("gluten SPARK-35650: Coalesce number of partitions by AEQ") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1") { + Seq("REPARTITION", "REBALANCE(key)") + .foreach { + repartition => + val query = s"SELECT /*+ $repartition */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 1) + assert(!aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + } + + test("gluten SPARK-35650: Use local shuffle read if can not coalesce number of partitions") { + withSQLConf(SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "false") { + val query = "SELECT /*+ REPARTITION */ * FROM testData" + val (_, adaptivePlan) = runAdaptiveAndVerifyResult(query) + collect(adaptivePlan) { case r: AQEShuffleReadExec => r } match { + case Seq(aqeShuffleRead) => + assert(aqeShuffleRead.partitionSpecs.size === 4) + assert(aqeShuffleRead.isLocalRead) + case _ => + fail("There should be a AQEShuffleReadExec") + } + } + } + + test("gluten SPARK-35725: Support optimize skewed partitions in RebalancePartitions") { + withTempView("v") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1" + ) { + + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(if (i > 4) 5 else i, i.toString)), 3) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + def checkPartitionNumber( + query: String, + skewedPartitionNumber: Int, + totalNumber: Int): Unit = { + val (_, adaptive) = runAdaptiveAndVerifyResult(query) + val read = collect(adaptive) { case read: AQEShuffleReadExec => read } + assert(read.size == 1) + assert( + read.head.partitionSpecs.count(_.isInstanceOf[PartialReducerPartitionSpec]) == + skewedPartitionNumber) + assert(read.head.partitionSpecs.size == totalNumber) + } + + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "150") { + // partition size [0,258,72,72,72] + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 3, 6) + // partition size [72,216,216,144,72] + checkPartitionNumber("SELECT /*+ REBALANCE */ * FROM v", 9, 10) + } + + // no skewed partition should be optimized + withSQLConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key -> "10000") { + checkPartitionNumber("SELECT /*+ REBALANCE(c1) */ * FROM v", 0, 1) + } + } + } + } + + test("gluten SPARK-35888: join with a 0-partition table") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_NUM.key -> "1", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + withTempView("t2") { + // create a temp view with 0 partition + spark + .createDataFrame(sparkContext.emptyRDD[Row], new StructType().add("b", IntegerType)) + .createOrReplaceTempView("t2") + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT * FROM testData2 t1 left semi join t2 ON t1.a=t2.b") + val aqeReads = collect(adaptive) { case c: AQEShuffleReadExec => c } + assert(aqeReads.length == 2) + aqeReads.foreach { + c => + val stats = c.child.asInstanceOf[QueryStageExec].getRuntimeStatistics + assert(stats.sizeInBytes >= 0) + assert(stats.rowCount.get >= 0) + } + } + } + } + + test("gluten SPARK-35968: AQE coalescing should not produce too small partitions by default") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptive) = + runAdaptiveAndVerifyResult("SELECT sum(id) FROM RANGE(10) GROUP BY id % 3") + val coalesceRead = collect(adaptive) { + case r: AQEShuffleReadExec if r.hasCoalescedPartition => r + } + assert(coalesceRead.length == 1) + // RANGE(10) is a very small dataset and AQE coalescing should produce one partition. + assert(coalesceRead.head.partitionSpecs.length == 1) + } + } + + test("gluten SPARK-35794: Allow custom plugin for cost evaluator") { + CostEvaluator.instantiate( + classOf[SimpleShuffleSortCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + intercept[IllegalArgumentException] { + CostEvaluator.instantiate( + classOf[InvalidCostEvaluator].getCanonicalName, + spark.sparkContext.getConf) + } + + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + val query = "SELECT * FROM testData join testData2 ON key = a where value = '1'" + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.SimpleShuffleSortCostEvaluator") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult(query) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.size == 1) + checkNumLocalShuffleReads(adaptivePlan) + } + + withSQLConf( + SQLConf.ADAPTIVE_CUSTOM_COST_EVALUATOR_CLASS.key -> + "org.apache.spark.sql.execution.adaptive.InvalidCostEvaluator") { + intercept[IllegalArgumentException] { + runAdaptiveAndVerifyResult(query) + } + } + } + } + + test("gluten SPARK-36020: Check logical link in remove redundant projects") { + withTempView("t") { + spark + .range(10) + .selectExpr( + "id % 10 as key", + "cast(id * 2 as int) as a", + "cast(id * 3 as int) as b", + "array(id, id + 1, id + 3) as c") + .createOrReplaceTempView("t") + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> "800") { + val query = + """ + |WITH tt AS ( + | SELECT key, a, b, explode(c) AS c FROM t + |) + |SELECT t1.key, t1.c, t2.key, t2.c + |FROM (SELECT a, b, c, key FROM tt WHERE a > 1) t1 + |JOIN (SELECT a, b, c, key FROM tt) t2 + | ON t1.key = t2.key + |""".stripMargin + val (origin, adaptive) = runAdaptiveAndVerifyResult(query) + assert(findTopLevelSortMergeJoin(origin).size == 1) + assert(findTopLevelBroadcastHashJoinTransform(adaptive).size == 1) + } + } + } + + test( + "gluten " + + "SPARK-36032: Use inputPlan instead of currentPhysicalPlan to initialize logical link") { + withTempView("v") { + spark.sparkContext + .parallelize((1 to 10).map(i => TestData(i, i.toString)), 2) + .toDF("c1", "c2") + .createOrReplaceTempView("v") + + Seq("-1", "10000").foreach { + aqeBhj => + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.ADAPTIVE_AUTO_BROADCASTJOIN_THRESHOLD.key -> aqeBhj, + SQLConf.SHUFFLE_PARTITIONS.key -> "1" + ) { + val (origin, adaptive) = runAdaptiveAndVerifyResult(""" + |SELECT * FROM v t1 JOIN ( + | SELECT c1 + 1 as c3 FROM v + |)t2 ON t1.c1 = t2.c3 + |SORT BY c1 + """.stripMargin) + if (aqeBhj.toInt < 0) { + // 1 sort since spark plan has no shuffle for SMJ + assert(findTopLevelSort(origin).size == 1) + // 2 sorts in SMJ + assert(findTopLevelSortTransform(adaptive).size == 2) + } else { + assert(findTopLevelSort(origin).size == 1) + // 1 sort at top node and BHJ has no sort + assert(findTopLevelSortTransform(adaptive).size == 1) + } + } + } + } + } + + test("gluten SPARK-37742: AQE reads invalid InMemoryRelation stats and mistakenly plans BHJ") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1048584", + SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> AQEPropagateEmptyRelation.ruleName + ) { + // Spark estimates a string column as 20 bytes so with 60k rows, these relations should be + // estimated at ~120m bytes which is greater than the broadcast join threshold. + val joinKeyOne = "00112233445566778899" + val joinKeyTwo = "11223344556677889900" + Seq + .fill(60000)(joinKeyOne) + .toDF("key") + .createOrReplaceTempView("temp") + Seq + .fill(60000)(joinKeyTwo) + .toDF("key") + .createOrReplaceTempView("temp2") + + Seq(joinKeyOne).toDF("key").createOrReplaceTempView("smallTemp") + spark.sql("SELECT key as newKey FROM temp").persist() + + // This query is trying to set up a situation where there are three joins. + // The first join will join the cached relation with a smaller relation. + // The first join is expected to be a broadcast join since the smaller relation will + // fit under the broadcast join threshold. + // The second join will join the first join with another relation and is expected + // to remain as a sort-merge join. + // The third join will join the cached relation with another relation and is expected + // to remain as a sort-merge join. + val query = + s""" + |SELECT t3.newKey + |FROM + | (SELECT t1.newKey + | FROM (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM smallTemp) as t2 + | ON t1.newKey = t2.key + | ) as t3 + | JOIN + | (SELECT key FROM temp2) as t4 + | ON t3.newKey = t4.key + |UNION + |SELECT t1.newKey + |FROM + | (SELECT key as newKey FROM temp) as t1 + | JOIN + | (SELECT key FROM temp2) as t2 + | ON t1.newKey = t2.key + |""".stripMargin + val df = spark.sql(query) + df.collect() + val adaptivePlan = df.queryExecution.executedPlan + val bhj = findTopLevelBroadcastHashJoinTransform(adaptivePlan) + assert(bhj.length == 1) + } + } + + ignore("gluten test log level") { + def verifyLog(expectedLevel: Level): Unit = { + val logAppender = new LogAppender("adaptive execution") + logAppender.setThreshold(expectedLevel) + withLogAppender( + logAppender, + loggerNames = Seq(AdaptiveSparkPlanExec.getClass.getName.dropRight(1)), + level = Some(Level.TRACE)) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "300") { + sql("SELECT * FROM testData join testData2 ON key = a where value = '1'").collect() + } + } + Seq("Plan changed", "Final plan").foreach { + msg => + assert(logAppender.loggingEvents.exists { + event => + event.getMessage.getFormattedMessage.contains(msg) && event.getLevel == expectedLevel + }) + } + } + + // Verify default log level + verifyLog(Level.DEBUG) + + // Verify custom log level + val levels = Seq( + "TRACE" -> Level.TRACE, + "trace" -> Level.TRACE, + "DEBUG" -> Level.DEBUG, + "debug" -> Level.DEBUG, + "INFO" -> Level.INFO, + "info" -> Level.INFO, + "WARN" -> Level.WARN, + "warn" -> Level.WARN, + "ERROR" -> Level.ERROR, + "error" -> Level.ERROR, + "deBUG" -> Level.DEBUG + ) + + levels.foreach { + level => + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_LOG_LEVEL.key -> level._1) { + verifyLog(level._2) + } + } + } +} 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 new file mode 100644 index 000000000000..87011d63ffb3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/benchmarks/ParquetReadBenchmark.scala @@ -0,0 +1,237 @@ +/* + * 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.benchmarks + +import io.glutenproject.GlutenConfig +import io.glutenproject.backendsapi.BackendsApiManager +import io.glutenproject.execution.{FileSourceScanExecTransformer, WholeStageTransformer} +import io.glutenproject.utils.{BackendTestUtils, SystemParameters} +import io.glutenproject.vectorized.JniLibLoader + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile} +import org.apache.spark.sql.vectorized.ColumnarBatch + +import scala.collection.JavaConverters._ + +/** + * Benchmark to measure native parquet read performance. To run this benchmark: + * {{{ + * 1. Run in IDEA: run this class directly; + * 2. Run without IDEA: bin/spark-submit --class + * --jars ,, + * --conf xxxx=xxx + * gluten-ut-XXX-tests.jar + * parameters + * + * Parameters: + * 1. parquet files dir; + * 2. the fields to read; + * 3. the execution count; + * 4. whether to run vanilla spark benchmarks; + * }}} + */ +object ParquetReadBenchmark extends SqlBasedBenchmark { + + protected lazy val thrdNum = "1" + protected lazy val memorySize = "4G" + protected lazy val offheapSize = "4G" + + def beforeAll(): Unit = {} + + override def getSparkSession: SparkSession = { + beforeAll(); + val conf = new SparkConf() + .setAppName("ParquetReadBenchmark") + .setIfMissing("spark.master", s"local[$thrdNum]") + .set("spark.plugins", "io.glutenproject.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .setIfMissing("spark.memory.offHeap.size", offheapSize) + .setIfMissing("spark.sql.columnVector.offheap.enabled", "true") + .set("spark.gluten.sql.columnar.columnarToRow", "true") + .set("spark.sql.adaptive.enabled", "false") + .setIfMissing("spark.driver.memory", memorySize) + .setIfMissing("spark.executor.memory", memorySize) + .setIfMissing("spark.sql.files.maxPartitionBytes", "1G") + .setIfMissing("spark.sql.files.openCostInBytes", "1073741824") + + if (BackendTestUtils.isCHBackendLoaded()) { + conf + .set("spark.io.compression.codec", "LZ4") + .set("spark.gluten.sql.enable.native.validation", "false") + .set("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .set("spark.gluten.sql.columnar.backend.ch.use.v2", "false") + .set("spark.gluten.sql.columnar.separate.scan.rdd.for.ch", "false") + .setIfMissing(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + .set( + "spark.sql.catalog.spark_catalog", + "org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseSparkCatalog") + .set("spark.databricks.delta.maxSnapshotLineageLength", "20") + .set("spark.databricks.delta.snapshotPartitions", "1") + .set("spark.databricks.delta.properties.defaults.checkpointInterval", "5") + .set("spark.databricks.delta.stalenessLimit", "3600000") + } + + SparkSession.builder.config(conf).getOrCreate() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val (parquetDir, scanSchema, executedCnt, executedVanilla) = + if (mainArgs.isEmpty) { + ("/data/tpch-data-sf10/lineitem", "l_orderkey,l_receiptdate", 5, true) + } else { + (mainArgs(0), mainArgs(1), mainArgs(2).toInt, mainArgs(3).toBoolean) + } + + val parquetReadDf = spark.sql(s""" + |select $scanSchema from parquet.`$parquetDir` + | + |""".stripMargin) + // Get the `FileSourceScanExecTransformer` + val fileScan = parquetReadDf.queryExecution.executedPlan.collect { + case scan: FileSourceScanExecTransformer => scan + }.head + + val filePartitions = fileScan.getPartitions + .map(_.asInstanceOf[FilePartition]) + + val wholeStageTransform = parquetReadDf.queryExecution.executedPlan.collect { + case wholeStage: WholeStageTransformer => wholeStage + }.head + + // remove ProjectExecTransformer + val newWholeStage = wholeStageTransform.withNewChildren(Seq(fileScan)) + + // generate ColumnarToRow + val columnarToRowPlan = + BackendsApiManager.getSparkPlanExecApiInstance.genColumnarToRowExec(newWholeStage) + + val newWholeStageRDD = newWholeStage.executeColumnar() + val newColumnarToRowRDD = columnarToRowPlan.execute() + + // Get the total row count + val totalRowCnt = newWholeStageRDD + .mapPartitionsInternal( + batches => { + batches.map(batch => batch.numRows().toLong) + }) + .collect() + .sum + + val parquetReadBenchmark = + new Benchmark( + s"Parquet Read files, fields: $scanSchema, total $totalRowCnt records", + totalRowCnt, + output = output) + + parquetReadBenchmark.addCase(s"Native Parquet Read", executedCnt) { + _ => + val resultRDD: RDD[Long] = newWholeStageRDD.mapPartitionsInternal { + batches => + batches.foreach(batch => batch.numRows().toLong) + Iterator.empty + } + resultRDD.collect() + } + + parquetReadBenchmark.addCase(s"Native Parquet Read to Rows", executedCnt) { + _ => + val resultRDD: RDD[Int] = newColumnarToRowRDD.mapPartitionsInternal { + rows => + rows.foreach(_.numFields) + Iterator.empty + } + resultRDD.collect() + } + + if (executedVanilla) { + spark.conf.set("spark.gluten.enabled", "false") + + val vanillaParquet = spark.sql(s""" + |select $scanSchema from parquet.`$parquetDir` + | + |""".stripMargin) + + val vanillaScanPlan = vanillaParquet.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan + } + + val fileScan = vanillaScanPlan.head + val fileScanOutput = fileScan.output + val relation = fileScan.relation + val readFile: (PartitionedFile) => Iterator[InternalRow] = + relation.fileFormat.buildReaderWithPartitionValues( + sparkSession = relation.sparkSession, + dataSchema = relation.dataSchema, + partitionSchema = relation.partitionSchema, + requiredSchema = fileScan.requiredSchema, + filters = Seq.empty, + options = relation.options, + hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) + ) + + val newFileScanRDD = new FileScanRDD(spark, readFile, filePartitions, fileScan.requiredSchema) + .asInstanceOf[RDD[ColumnarBatch]] + + val rowCnt = newFileScanRDD + .mapPartitionsInternal(batches => batches.map(batch => batch.numRows().toLong)) + .collect() + .sum + assert(totalRowCnt == rowCnt, "The row count of the benchmark is not equal.") + + parquetReadBenchmark.addCase(s"Vanilla Spark Parquet Read", executedCnt) { + _ => + val resultRDD: RDD[Long] = newFileScanRDD.mapPartitionsInternal { + batches => + batches.foreach(_.numRows().toLong) + Iterator.empty + } + resultRDD.collect() + } + + parquetReadBenchmark.addCase(s"Vanilla Spark Parquet Read to Rows", executedCnt) { + _ => + val resultRDD: RDD[Long] = newFileScanRDD.mapPartitionsInternal { + batches => + val toUnsafe = UnsafeProjection.create(fileScanOutput, fileScanOutput) + batches.foreach(_.rowIterator().asScala.map(toUnsafe).foreach(_.numFields)) + Iterator.empty + } + resultRDD.collect() + } + } + + parquetReadBenchmark.run() + } + + override def afterAll(): Unit = { + if (BackendTestUtils.isCHBackendLoaded()) { + val libPath = + spark.conf.get(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + JniLibLoader.unloadFromPath(libPath) + } + super.afterAll() + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala new file mode 100644 index 000000000000..37a786e34c53 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenBucketingUtilsSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketingUtilsSuite extends BucketingUtilsSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala new file mode 100644 index 000000000000..eeb63436c1e1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceStrategySuite extends DataSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala new file mode 100644 index 000000000000..6435d17de2ab --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceSuite extends DataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.scala new file mode 100644 index 000000000000..664caf560499 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileFormatWriterSuite.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.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest + +class GlutenFileFormatWriterSuite + extends FileFormatWriterSuite + with GlutenSQLTestsBaseTrait + with CodegenInterpretedPlanTest { + + test("gluten empty file should be skipped while write to file") { + withTempPath { + path => + spark.range(100).repartition(10).where("id = 50").write.parquet(path.toString) + val partFiles = path + .listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + // result only one row, gluten result is more reasonable + assert(partFiles.length === 1) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala new file mode 100644 index 000000000000..c1c57eaa9145 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileIndexSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileIndexSuite extends FileIndexSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala new file mode 100644 index 000000000000..af15f7386fca --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileMetadataStructSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileMetadataStructSuite extends FileMetadataStructSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.scala new file mode 100644 index 000000000000..54138564f95f --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceAggregatePushDownSuite.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.spark.sql.execution.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetV1AggregatePushDownSuite + extends ParquetV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetV2AggregatePushDownSuite + extends ParquetV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1AggregatePushDownSuite + extends OrcV1AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV2AggregatePushDownSuite + extends OrcV2AggregatePushDownSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala new file mode 100644 index 000000000000..631be9c96fa9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceCodecSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetCodecSuite extends ParquetCodecSuite with GlutenSQLTestsBaseTrait {} + +class GlutenOrcCodecSuite extends OrcCodecSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala new file mode 100644 index 000000000000..171a27e31c47 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenFileSourceStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql._ + +class GlutenFileSourceStrategySuite extends FileSourceStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala new file mode 100644 index 000000000000..b283d44b03a4 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenHadoopFileLinesReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenHadoopFileLinesReaderSuite + extends HadoopFileLinesReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala new file mode 100644 index 000000000000..f3554eb1cb09 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterStrategySuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterStrategySuite extends PathFilterStrategySuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala new file mode 100644 index 000000000000..4f4f9c76ee4a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPathFilterSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathFilterSuite extends PathFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala new file mode 100644 index 000000000000..a108c4fe1ecf --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenPruneFileSourcePartitionsSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPruneFileSourcePartitionsSuite + extends PruneFileSourcePartitionsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala new file mode 100644 index 000000000000..982e1bc4e29c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/GlutenReadSchemaSuite.scala @@ -0,0 +1,144 @@ +/* + * 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.datasources + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants} +import org.apache.spark.sql.internal.SQLConf + +import java.io.File + +class GlutenCSVReadSchemaSuite extends CSVReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenHeaderCSVReadSchemaSuite + extends HeaderCSVReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenJsonReadSchemaSuite extends JsonReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenOrcReadSchemaSuite extends OrcReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedOrcReadSchemaSuite + extends VectorizedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + private lazy val values = 1 to 10 + private lazy val floatDF = values.map(_.toFloat).toDF("col1") + private lazy val doubleDF = values.map(_.toDouble).toDF("col1") + private lazy val unionDF = floatDF.union(doubleDF) + + test(GlutenTestConstants.GLUTEN_TEST + "change column position") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val df1 = Seq(("1", "a"), ("2", "b"), ("3", "c")).toDF("col1", "col2") + val df2 = Seq(("d", "4"), ("e", "5"), ("f", "6")).toDF("col2", "col1") + val unionDF = df1.unionByName(df2) + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1", "col2") + + checkAnswer(df, unionDF) + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "read byte, int, short, long together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val byteDF = (Byte.MaxValue - 2 to Byte.MaxValue).map(_.toByte).toDF("col1") + val shortDF = (Short.MaxValue - 2 to Short.MaxValue).map(_.toShort).toDF("col1") + val intDF = (Int.MaxValue - 2 to Int.MaxValue).toDF("col1") + val longDF = (Long.MaxValue - 2 to Long.MaxValue).toDF("col1") + val unionDF = byteDF.union(shortDF).union(intDF).union(longDF) + + val byteDir = s"$path${File.separator}part=byte" + val shortDir = s"$path${File.separator}part=short" + val intDir = s"$path${File.separator}part=int" + val longDir = s"$path${File.separator}part=long" + + byteDF.write.format(format).options(options).save(byteDir) + shortDF.write.format(format).options(options).save(shortDir) + intDF.write.format(format).options(options).save(intDir) + longDF.write.format(format).options(options).save(longDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "read float and double together") { + withTempPath { + dir => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { + val path = dir.getCanonicalPath + + val floatDir = s"$path${File.separator}part=float" + val doubleDir = s"$path${File.separator}part=double" + + floatDF.write.format(format).options(options).save(floatDir) + doubleDF.write.format(format).options(options).save(doubleDir) + + val df = spark.read + .schema(unionDF.schema) + .format(format) + .options(options) + .load(path) + .select("col1") + + checkAnswer(df, unionDF) + } + } + } +} + +class GlutenMergedOrcReadSchemaSuite + extends MergedOrcReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetReadSchemaSuite extends ParquetReadSchemaSuite with GlutenSQLTestsBaseTrait {} + +class GlutenVectorizedParquetReadSchemaSuite + extends VectorizedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenMergedParquetReadSchemaSuite + extends MergedParquetReadSchemaSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala new file mode 100644 index 000000000000..ee6ec1bea1af --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/GlutenBinaryFileFormatSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.binaryfile + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBinaryFileFormatSuite extends BinaryFileFormatSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.scala new file mode 100644 index 000000000000..4181a32521cf --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/csv/GlutenCSVSuite.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.spark.sql.execution.datasources.csv + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf + +class GlutenCSVSuite extends CSVSuite with GlutenSQLTestsBaseTrait { + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenCSVv1Suite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "csv") +} + +class GlutenCSVv2Suite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} + +class GlutenCSVLegacyTimeParserSuite extends GlutenCSVSuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, "legacy") +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.scala new file mode 100644 index 000000000000..132e80696cff --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/exchange/GlutenValidateRequirementsSuite.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.execution.datasources.exchange + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.execution.exchange.ValidateRequirementsSuite + +class GlutenValidateRequirementsSuite + extends ValidateRequirementsSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala new file mode 100644 index 000000000000..4b7e3cc54e8d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/json/GlutenJsonSuite.scala @@ -0,0 +1,82 @@ +/* + * 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.datasources.json + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{sources, GlutenSQLTestsBaseTrait} +import org.apache.spark.sql.execution.datasources.{InMemoryFileIndex, NoopCache} +import org.apache.spark.sql.execution.datasources.v2.json.JsonScanBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class GlutenJsonSuite extends JsonSuite with GlutenSQLTestsBaseTrait { + + /** Returns full path to the given file in the resource folder */ + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } +} + +class GlutenJsonV1Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "json") +} + +class GlutenJsonV2Suite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + test("get pushed filters") { + val attr = "col" + def getBuilder(path: String): JsonScanBuilder = { + val fileIndex = new InMemoryFileIndex( + spark, + Seq(new org.apache.hadoop.fs.Path(path, "file.json")), + Map.empty, + None, + NoopCache) + val schema = new StructType().add(attr, IntegerType) + val options = CaseInsensitiveStringMap.empty() + new JsonScanBuilder(spark, fileIndex, schema, schema, options) + } + val filters: Array[sources.Filter] = Array(sources.IsNotNull(attr)) + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === filters) + } + } + + withSQLConf(SQLConf.JSON_FILTER_PUSHDOWN_ENABLED.key -> "false") { + withTempPath { + file => + val scanBuilder = getBuilder(file.getCanonicalPath) + assert(scanBuilder.pushDataFilters(filters) === Array.empty[sources.Filter]) + } + } + } +} + +class GlutenJsonLegacyTimeParserSuite extends GlutenJsonSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.LEGACY_TIME_PARSER_POLICY, "legacy") +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala new file mode 100644 index 000000000000..e2e3818aad9c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcColumnarBatchReaderSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcColumnarBatchReaderSuite + extends OrcColumnarBatchReaderSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala new file mode 100644 index 000000000000..f5a8db3395d6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcFilterSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** A test suite that tests Apache ORC filter API based filter pushdown optimization. */ +class GlutenOrcFilterSuite extends OrcFilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..a9848b7f444d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcPartitionDiscoverySuite.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.sql.execution.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcPartitionDiscoverySuite + extends OrcPartitionDiscoveryTest + with GlutenSQLTestsBaseTrait {} + +class GlutenOrcV1PartitionDiscoverySuite + extends OrcV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala new file mode 100644 index 000000000000..2148f36de584 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcQuerySuite.scala @@ -0,0 +1,160 @@ +/* + * 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.datasources.orc + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants, Row} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.internal.SQLConf + +class GlutenOrcQuerySuite extends OrcQuerySuite with GlutenSQLTestsBaseTrait { + test(GlutenTestConstants.GLUTEN_TEST + "Simple selection form ORC table") { + val data = (1 to 10).map { + i => Person(s"name_$i", i, (0 to 1).map(m => Contact(s"contact_$m", s"phone_$m"))) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = leaf-0 + assert(sql("SELECT name FROM t WHERE age <= 5").count() === 5) + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // expr = (not leaf-0) + assertResult(10) { + sql("SELECT name, contacts FROM t where age > 5").rdd + .flatMap(_.getAs[scala.collection.Seq[_]]("contacts")) + .count() + } + + // ppd: + // leaf-0 = (LESS_THAN_EQUALS age 5) + // leaf-1 = (LESS_THAN age 8) + // expr = (and (not leaf-0) leaf-1) + { + val df = sql("SELECT name, contacts FROM t WHERE age > 5 AND age < 8") + assert(df.count() === 2) + assertResult(4) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + + // ppd: + // leaf-0 = (LESS_THAN age 2) + // leaf-1 = (LESS_THAN_EQUALS age 8) + // expr = (or leaf-0 (not leaf-1)) + { + val df = sql("SELECT name, contacts FROM t WHERE age < 2 OR age > 8") + assert(df.count() === 3) + assertResult(6) { + df.rdd.flatMap(_.getAs[scala.collection.Seq[_]]("contacts")).count() + } + } + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "simple select queries") { + withOrcTable((0 until 10).map(i => (i, i.toString)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t where t.`_1` > 5"), (6 until 10).map(Row.apply(_))) + + checkAnswer( + sql("SELECT `_1` FROM t as tmp where tmp.`_1` < 5"), + (0 until 5).map(Row.apply(_))) + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "overwriting") { + val data = (0 until 10).map(i => (i, i.toString)) + spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + sql("INSERT OVERWRITE TABLE t SELECT * FROM tmp") + checkAnswer(spark.table("t"), data.map(Row.fromTuple)) + } + } + spark.sessionState.catalog.dropTable( + TableIdentifier("tmp"), + ignoreIfNotExists = true, + purge = false) + } + + test(GlutenTestConstants.GLUTEN_TEST + "self-join") { + // 4 rows, cells of column 1 of row 2 and row 4 are null + val data = (1 to 4).map { + i => + val maybeInt = if (i % 2 == 0) None else Some(i) + (maybeInt, i.toString) + } + + withOrcTable(data, "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + val selfJoin = sql("SELECT * FROM t x JOIN t y WHERE x.`_1` = y.`_1`") + val queryOutput = selfJoin.queryExecution.analyzed.output + + assertResult(4, "Field count mismatches")(queryOutput.size) + assertResult(2, s"Duplicated expression ID in query plan:\n $selfJoin") { + queryOutput.filter(_.name == "_1").map(_.exprId).size + } + + checkAnswer(selfJoin, List(Row(1, "1", 1, "1"), Row(3, "3", 3, "3"))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "columns only referenced by pushed down filters should remain") { + withOrcTable((1 to 10).map(Tuple1.apply), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer(sql("SELECT `_1` FROM t WHERE `_1` < 10"), (1 to 9).map(Row.apply(_))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-5309 strings stored using dictionary compression in orc") { + withOrcTable((0 until 1000).map(i => ("same", "run_" + i / 100, 1)), "t") { + withSQLConf("spark.sql.orc.enableVectorizedReader" -> "false") { + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t GROUP BY `_1`, `_2`"), + (0 until 10).map(i => Row("same", "run_" + i, 100))) + + checkAnswer( + sql("SELECT `_1`, `_2`, SUM(`_3`) FROM t WHERE `_2` = 'run_5' GROUP BY `_1`, `_2`"), + List(Row("same", "run_5", 100))) + } + } + } +} + +class GlutenOrcV1QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "orc") +} + +class GlutenOrcV2QuerySuite extends GlutenOrcQuerySuite { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.scala new file mode 100644 index 000000000000..7535ee563fae --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcSourceSuite.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.datasources.orc + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants, Row} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DayTimeIntervalType, IntegerType, StructField, StructType, YearMonthIntervalType} + +import java.sql.Date +import java.time.{Duration, Period} + +class GlutenOrcSourceSuite extends OrcSourceSuite with GlutenSQLTestsBaseTrait { + import testImplicits._ + + override def withAllNativeOrcReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false")(code) + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-31238: compatibility with Spark 2.4 in reading dates") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_date_v2_4.snappy.orc"), + Row(java.sql.Date.valueOf("1200-01-01"))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-31238, SPARK-31423: rebasing dates in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01", "1582-10-10") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq(Row(Date.valueOf("1001-01-01")), Row(Date.valueOf("1582-10-15")))) + } + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-31284: compatibility with Spark 2.4 in reading timestamps") { + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + readResourceOrcFile("test-data/before_1582_ts_v2_4.snappy.orc"), + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456"))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-31284, SPARK-31423: rebasing timestamps in write") { + withTempPath { + dir => + val path = dir.getAbsolutePath + Seq("1001-01-01 01:02:03.123456", "1582-10-10 11:12:13.654321") + .toDF("tsS") + .select($"tsS".cast("timestamp").as("ts")) + .write + .orc(path) + + Seq(false).foreach { + vectorized => + withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + checkAnswer( + spark.read.orc(path), + Seq( + Row(java.sql.Timestamp.valueOf("1001-01-01 01:02:03.123456")), + Row(java.sql.Timestamp.valueOf("1582-10-15 11:12:13.654321")))) + } + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-34862: Support ORC vectorized reader for nested column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val df = spark + .range(10) + .map { + x => + val stringColumn = s"$x" * 10 + val structColumn = (x, s"$x" * 100) + val arrayColumn = (0 until 5).map(i => (x + i, s"$x" * 5)) + val mapColumn = Map( + s"$x" -> (x * 0.1, (x, s"$x" * 100)), + (s"$x" * 2) -> (x * 0.2, (x, s"$x" * 200)), + (s"$x" * 3) -> (x * 0.3, (x, s"$x" * 300))) + (x, stringColumn, structColumn, arrayColumn, mapColumn) + } + .toDF("int_col", "string_col", "struct_col", "array_col", "map_col") + df.write.format("orc").save(path) + + // Rewrite because Gluten does not support Spark's vectorized reading. + withSQLConf(SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> "false") { + val readDf = spark.read.orc(path) + val vectorizationEnabled = readDf.queryExecution.executedPlan.find { + case scan: FileSourceScanExec => scan.supportsColumnar + case _ => false + }.isDefined + assert(!vectorizationEnabled) + checkAnswer(readDf, df) + } + } + } + withAllNativeOrcReaders { + Seq(false).foreach { + vecReaderNestedColEnabled => + val vecReaderEnabled = SQLConf.get.orcVectorizedReaderEnabled + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-36931: Support reading and writing ANSI intervals (" + + s"${SQLConf.ORC_VECTORIZED_READER_ENABLED.key}=$vecReaderEnabled, " + + s"${SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key}" + + s"=$vecReaderNestedColEnabled)") { + + withSQLConf( + SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> + vecReaderEnabled.toString, + SQLConf.ORC_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key -> + vecReaderNestedColEnabled.toString + ) { + Seq( + YearMonthIntervalType() -> ((i: Int) => Period.of(i, i, 0)), + DayTimeIntervalType() -> ((i: Int) => Duration.ofDays(i).plusSeconds(i)) + ).foreach { + case (it, f) => + val data = (1 to 10).map(i => Row(i, f(i))) + val schema = StructType( + Array(StructField("d", IntegerType, false), StructField("i", it, false))) + withTempPath { + file => + val df = spark.createDataFrame(sparkContext.parallelize(data), schema) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + + // Tests for ANSI intervals in complex types. + withTempPath { + file => + val df = spark.sql("""SELECT + | named_struct('interval', interval '1-2' year to month) a, + | array(interval '1 2:3' day to minute) b, + | map('key', interval '10' year) c, + | map(interval '20' second, 'value') d""".stripMargin) + df.write.orc(file.getCanonicalPath) + val df2 = spark.read.orc(file.getCanonicalPath) + checkAnswer(df2, df.collect().toSeq) + } + } + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala new file mode 100644 index 000000000000..3c2fb0b318f1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1FilterSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOrcV1FilterSuite extends OrcV1FilterSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala new file mode 100644 index 000000000000..90ed84ec2a8d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV1SchemaPruningSuite.scala @@ -0,0 +1,28 @@ +/* + * 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.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV1SchemaPruningSuite extends OrcV1SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala new file mode 100644 index 000000000000..9d758201fd19 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/orc/GlutenOrcV2SchemaPruningSuite.scala @@ -0,0 +1,28 @@ +/* + * 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.datasources.orc + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenOrcV2SchemaPruningSuite extends OrcV2SchemaPruningSuite with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.scala new file mode 100644 index 000000000000..e69577fc621a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetColumnIndexSuite.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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} + +class GlutenParquetColumnIndexSuite extends ParquetColumnIndexSuite with GlutenSQLTestsBaseTrait { + private val actions: Seq[DataFrame => DataFrame] = Seq( + "_1 = 500", + "_1 = 500 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1500", + "_1 = 500 or _1 = 501 or _1 = 1000 or _1 = 1500", + "_1 >= 500 and _1 < 1000", + "(_1 >= 500 and _1 < 1000) or (_1 >= 1500 and _1 < 1600)" + ).map(f => (df: DataFrame) => df.filter(f)) + + test("Gluten: test reading unaligned pages - test all types") { + val df = spark + .range(0, 2000) + .selectExpr( + "id as _1", + "cast(id as short) as _3", + "cast(id as int) as _4", + "cast(id as float) as _5", + "cast(id as double) as _6", + "cast(id as decimal(20,0)) as _7", + "cast(cast(1618161925000 + id * 1000 * 60 * 60 * 24 as timestamp) as date) as _9" + ) + checkUnalignedPages(df)(actions: _*) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala new file mode 100644 index 000000000000..661d6aad8c3c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetCompressionCodecPrecedenceSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetCompressionCodecPrecedenceSuite + extends ParquetCompressionCodecPrecedenceSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..166f3255efd5 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.scala new file mode 100644 index 000000000000..ccb69819a3a3 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaEncodingSuite.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.sql.execution.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaEncodingInteger + extends ParquetDeltaEncodingInteger + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetDeltaEncodingLong + extends ParquetDeltaEncodingLong + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala new file mode 100644 index 000000000000..36928cee001d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetDeltaLengthByteArrayEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetDeltaLengthByteArrayEncodingSuite + extends ParquetDeltaLengthByteArrayEncodingSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala new file mode 100644 index 000000000000..6c69c700becc --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetEncodingSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// TODO: this needs a lot more testing but it's currently not easy to test with the parquet +// writer abstractions. Revisit. +class GlutenParquetEncodingSuite extends ParquetEncodingSuite with GlutenSQLTestsBaseTrait {} 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 new file mode 100644 index 000000000000..9e4d94e1c298 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFieldIdIOSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetFieldIdIOSuite extends ParquetFieldIdIOSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala new file mode 100644 index 000000000000..229547171724 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFileFormatSuite.scala @@ -0,0 +1,40 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf + +class GlutenParquetFileFormatV1Suite extends ParquetFileFormatV1Suite with GlutenSQLTestsBaseTrait { + override def withAllParquetReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")(code) + // Disabled: We don't yet support this case as of now + // test the vectorized reader + // withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true")(code) + } +} + +class GlutenParquetFileFormatV2Suite extends ParquetFileFormatV2Suite with GlutenSQLTestsBaseTrait { + override def withAllParquetReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")(code) + // Disabled: We don't yet support this case as of now + // test the vectorized reader + // withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true")(code) + } +} 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 new file mode 100644 index 000000000000..2532436aa206 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetFilterSuite.scala @@ -0,0 +1,637 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +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 +import org.apache.spark.sql.types._ +import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.Utils + +import org.apache.hadoop.fs.Path +import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate, Operators} +import org.apache.parquet.filter2.predicate.FilterApi._ +import org.apache.parquet.filter2.predicate.Operators +import org.apache.parquet.filter2.predicate.Operators.{Column => _, Eq, Gt, GtEq, Lt, LtEq, NotEq} +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetOutputFormat} +import org.apache.parquet.hadoop.util.HadoopInputFile + +import java.sql.{Date, Timestamp} +import java.time.LocalDate + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +abstract class GltuenParquetFilterSuite extends ParquetFilterSuite with GlutenSQLTestsBaseTrait { + protected def checkFilterPredicate( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: Seq[Row])(implicit df: DataFrame): Unit = { + checkFilterPredicate(df, predicate, filterClass, checkAnswer(_, _: Seq[Row]), expected) + } + + protected def checkFilterPredicate[T]( + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + expected: T)(implicit df: DataFrame): Unit = { + checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) + } + + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) + } + + test(GlutenTestConstants.GLUTEN_TEST + "filter pushdown - timestamp") { + Seq(true, false).foreach { + java8Api => + Seq(CORRECTED, LEGACY).foreach { + rebaseMode => + val millisData = Seq( + "1000-06-14 08:28:53.123", + "1582-06-15 08:28:53.001", + "1900-06-16 08:28:53.0", + "2018-06-17 08:28:53.999") + // INT96 doesn't support pushdown + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_INT96_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString, + SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> INT96.toString + ) { + import testImplicits._ + withTempPath { + file => + millisData + .map(i => Tuple1(Timestamp.valueOf(i))) + .toDF + .write + .format(dataSourceName) + .save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath) { + df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + assertResult(None) { + createParquetFilters(schema).createFilter(sources.IsNull("_1")) + } + } + } + } + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "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))) + } + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-12218: 'Not' is included in Parquet filter pushdown") { + import testImplicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { + dir => + val path = s"${dir.getCanonicalPath}/table1" + val df = (1 to 5).map(i => (i, (i % 2).toString)).toDF("a", "b") + df.show() + df.write.parquet(path) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2) or not(b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + + checkAnswer( + spark.read.parquet(path).where("not (a = 2 and b in ('1'))"), + (1 to 5).map(i => Row(i, (i % 2).toString))) + } + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-23852: Broken Parquet push-down for partially-written stats") { + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + // parquet-1217.parquet contains a single column with values -1, 0, 1, 2 and null. + // The row-group statistics include null counts, but not min and max values, which + // triggers PARQUET-1217. + + val df = readResourceParquetFile("test-data/parquet-1217.parquet") + + // Will return 0 rows if PARQUET-1217 is not fixed. + assert(df.where("col > 0").count() === 2) + } + } + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-17091: Convert IN predicate to Parquet filter push-down") { + val schema = StructType( + Seq( + StructField("a", IntegerType, nullable = false) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + assertResult(Some(FilterApi.eq(intColumn("a"), null: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(null))) + } + + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10))) + } + + // Remove duplicates + assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { + parquetFilters.createFilter(sources.In("a", Array(10, 10))) + } + + assertResult( + Some( + or( + or(FilterApi.eq(intColumn("a"), 10: Integer), FilterApi.eq(intColumn("a"), 20: Integer)), + FilterApi.eq(intColumn("a"), 30: Integer)))) { + parquetFilters.createFilter(sources.In("a", Array(10, 20, 30))) + } + + Seq(0, 10).foreach { + threshold => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> threshold.toString) { + assert( + createParquetFilters(parquetSchema) + .createFilter(sources.In("a", Array(10, 20, 30))) + .nonEmpty === threshold > 0) + } + } + + import testImplicits._ + withTempPath { + path => + val data = 0 to 1024 + data + .toDF("a") + .selectExpr("if (a = 1024, null, a) AS a") // convert 1024 to null + .coalesce(1) + .write + .option("parquet.block.size", 512) + .parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + Seq(true, false).foreach { + pushEnabled => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> pushEnabled.toString) { + Seq(1, 5, 10, 11, 100).foreach { + count => + val filter = s"a in(${Range(0, count).mkString(",")})" + assert(df.where(filter).count() === count) + val actual = stripSparkFilter(df.where(filter)).collect().length + assert(actual === count) + } + assert(df.where("a in(null)").count() === 0) + assert(df.where("a = null").count() === 0) + assert(df.where("a is null").count() === 1) + } + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "Support Parquet column index") { + // block 1: + // null count min max + // page-0 0 0 99 + // page-1 0 100 199 + // page-2 0 200 299 + // page-3 0 300 399 + // page-4 0 400 449 + // + // block 2: + // null count min max + // page-0 0 450 549 + // page-1 0 550 649 + // page-2 0 650 749 + // page-3 0 750 849 + // page-4 0 850 899 + withTempPath { + path => + spark + .range(900) + .repartition(1) + .write + .option(ParquetOutputFormat.PAGE_SIZE, "500") + .option(ParquetOutputFormat.BLOCK_SIZE, "2000") + .parquet(path.getCanonicalPath) + + val parquetFile = path.listFiles().filter(_.getName.startsWith("part")).last + val in = HadoopInputFile.fromPath( + new Path(parquetFile.getCanonicalPath), + spark.sessionState.newHadoopConf()) + + Utils.tryWithResource(ParquetFileReader.open(in)) { + reader => + val blocks = reader.getFooter.getBlocks + assert(blocks.size() > 1) + val columns = blocks.get(0).getColumns + assert(columns.size() === 1) + val columnIndex = reader.readColumnIndex(columns.get(0)) + assert(columnIndex.getMinValues.size() > 1) + + val rowGroupCnt = blocks.get(0).getRowCount + // Page count = Second page min value - first page min value + val pageCnt = columnIndex.getMinValues.get(1).asLongBuffer().get() - + columnIndex.getMinValues.get(0).asLongBuffer().get() + assert(pageCnt < rowGroupCnt) + Seq(true, false).foreach { + columnIndex => + withSQLConf(ParquetInputFormat.COLUMN_INDEX_FILTERING_ENABLED -> s"$columnIndex") { + val df = spark.read.parquet(parquetFile.getCanonicalPath).where("id = 1") + df.collect() + val plan = df.queryExecution.executedPlan + // Ignore metrics comparison. + /* + val metrics = plan.collectLeaves().head.metrics + val numOutputRows = metrics("numOutputRows").value + + if (columnIndex) { + assert(numOutputRows === pageCnt) + } else { + assert(numOutputRows === rowGroupCnt) + } + */ + } + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV1FilterSuite extends GltuenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "parquet") + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + Seq(("parquet", true), ("", false)).foreach { + case (pushdownDsList, nestedPredicatePushdown) => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.NESTED_PREDICATE_PUSHDOWN_FILE_SOURCE_LIST.key -> pushdownDsList + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + val nestedOrAttributes = predicate.collectFirst { + case g: GetStructField => g + case a: Attribute => a + } + assert(nestedOrAttributes.isDefined, "No GetStructField nor Attribute is detected.") + + val parsed = + parseColumnPath(PushableColumnAndNestedColumn.unapply(nestedOrAttributes.get).get) + + val containsNestedColumnOrDot = parsed.length > 1 || parsed(0).contains(".") + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan + .collect { + case PhysicalOperation( + _, + filters, + LogicalRelation(relation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(relation) + filters + } + .flatten + .reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + // If predicates contains nested column or dot, we push down the predicates only if + // "parquet" is in `NESTED_PREDICATE_PUSHDOWN_V1_SOURCE_LIST`. + if (nestedPredicatePushdown || !containsNestedColumnOrDot) { + assert(selectedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(selectedFilters) === selectedFilters) + val pushedParquetFilters = selectedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + } else { + assert(selectedFilters.isEmpty, "There is filter pushed down") + } + } + } + } +} + +@ExtendedSQLTest +class GlutenParquetV2FilterSuite extends GltuenParquetFilterSuite with GlutenSQLTestsBaseTrait { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false" + ) { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan.collectFirst { + case PhysicalOperation( + _, + filters, + DataSourceV2ScanRelation(_, scan: ParquetScan, _, None, None)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter(_, true)).toArray + val pushedFilters = scan.pushedFilters + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(sourceFilters) === pushedFilters) + val pushedParquetFilters = pushedFilters.map { + pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert( + pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain $filterClass.") + + checker(stripSparkFilter(query), expected) + + case _ => + throw new AnalysisException("Can not match ParquetTable in the query.") + } + } + } + + /** + * Takes a sequence of products `data` to generate multi-level nested dataframes as new test data. + * It tests both non-nested and nested dataframes which are written and read back with Parquet + * datasource. + * + * This is different from [[ParquetTest.withParquetDataFrame]] which does not test nested cases. + */ + private def withNestedParquetDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T])( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = + withNestedParquetDataFrame(spark.createDataFrame(data))(runTest) + + private def withNestedParquetDataFrame(inputDF: DataFrame)( + runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + withNestedDataFrame(inputDF).foreach { + case (newDF, colName, resultFun) => + withTempPath { + file => + newDF.write.format(dataSourceName).save(file.getCanonicalPath) + readParquetFile(file.getCanonicalPath)(df => runTest(df, colName, resultFun)) + } + } + } + + test(GlutenTestConstants.GLUTEN_TEST + "filter pushdown - date") { + implicit class StringToDate(s: String) { + def date: Date = Date.valueOf(s) + } + + val data = Seq("1000-01-01", "2018-03-19", "2018-03-20", "2018-03-21") + import testImplicits._ + + // Velox backend does not support rebaseMode being LEGACY. + Seq(false, true).foreach { + java8Api => + Seq(CORRECTED).foreach { + rebaseMode => + withSQLConf( + SQLConf.DATETIME_JAVA8API_ENABLED.key -> java8Api.toString, + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> rebaseMode.toString) { + val dates = data.map(i => Tuple1(Date.valueOf(i))).toDF() + withNestedParquetDataFrame(dates) { + case (inputDF, colName, fun) => + implicit val df: DataFrame = inputDF + + def resultFun(dateStr: String): Any = { + val parsed = if (java8Api) LocalDate.parse(dateStr) else Date.valueOf(dateStr) + fun(parsed) + } + + val dateAttr: Expression = df(colName).expr + assert(df(colName).expr.dataType === DateType) + + checkFilterPredicate(dateAttr.isNull, classOf[Eq[_]], Seq.empty[Row]) + checkFilterPredicate( + dateAttr.isNotNull, + classOf[NotEq[_]], + data.map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr === "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + logWarning(s"java8Api: $java8Api, rebaseMode, $rebaseMode") + checkFilterPredicate( + dateAttr <=> "1000-01-01".date, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr =!= "1000-01-01".date, + classOf[NotEq[_]], + Seq("2018-03-19", "2018-03-20", "2018-03-21").map(i => Row.apply(resultFun(i)))) + + checkFilterPredicate( + dateAttr < "2018-03-19".date, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr > "2018-03-20".date, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr <= "1000-01-01".date, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + dateAttr >= "2018-03-21".date, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + Literal("1000-01-01".date) === dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("1000-01-01".date) <=> dateAttr, + classOf[Eq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-19".date) > dateAttr, + classOf[Lt[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-20".date) < dateAttr, + classOf[Gt[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + Literal("1000-01-01".date) >= dateAttr, + classOf[LtEq[_]], + resultFun("1000-01-01")) + checkFilterPredicate( + Literal("2018-03-21".date) <= dateAttr, + classOf[GtEq[_]], + resultFun("2018-03-21")) + + checkFilterPredicate( + !(dateAttr < "2018-03-21".date), + classOf[GtEq[_]], + resultFun("2018-03-21")) + checkFilterPredicate( + dateAttr < "2018-03-19".date || dateAttr > "2018-03-20".date, + classOf[Operators.Or], + Seq(Row(resultFun("1000-01-01")), Row(resultFun("2018-03-21")))) + + Seq(3, 20).foreach { + threshold => + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_INFILTERTHRESHOLD.key -> s"$threshold") { + checkFilterPredicate( + In( + dateAttr, + Array( + "2018-03-19".date, + "2018-03-20".date, + "2018-03-21".date, + "2018-03-22".date).map(Literal.apply)), + if (threshold == 3) classOf[Operators.And] else classOf[Operators.Or], + Seq( + Row(resultFun("2018-03-19")), + Row(resultFun("2018-03-20")), + Row(resultFun("2018-03-21"))) + ) + } + } + } + } + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala new file mode 100644 index 000000000000..09bc0fc39ee1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetIOSuite.scala @@ -0,0 +1,43 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf + +/** A test suite that tests basic Parquet I/O. */ +class GlutenParquetIOSuite extends ParquetIOSuite with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key + "_DISABLED" + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + + override protected def testFile(fileName: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + fileName + } + override def withAllParquetReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + withClue("Parquet-mr reader") { + code + } + } + } + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet(testFile(name)) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala new file mode 100644 index 000000000000..051343dafb06 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetInteroperabilitySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetInteroperabilitySuite + extends ParquetInteroperabilitySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala new file mode 100644 index 000000000000..96a0b1e3a954 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetPartitionDiscoverySuite.scala @@ -0,0 +1,319 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.TimestampTypes +import org.apache.spark.sql.types.{ByteType, DateType, DecimalType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructField, StructType} + +import java.math.BigInteger +import java.sql.Timestamp +import java.time.LocalDateTime + +class GlutenParquetV1PartitionDiscoverySuite + extends ParquetV1PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait { + test("gluten: Various partition value types") { + Seq(TimestampTypes.TIMESTAMP_NTZ).foreach { + tsType => + withSQLConf(SQLConf.TIMESTAMP_TYPE.key -> tsType.toString) { + val ts = if (tsType == TimestampTypes.TIMESTAMP_LTZ) { + new Timestamp(0) + } else { + LocalDateTime.parse("1970-01-01T00:00:00") + } + val row = + Row( + 100.toByte, + 40000.toShort, + Int.MaxValue, + Long.MaxValue, + 1.5.toFloat, + 4.5, + new java.math.BigDecimal(new BigInteger("212500"), 5), + new java.math.BigDecimal("2.125"), + java.sql.Date.valueOf("2015-05-23"), + ts, + "This is a string, /[]?=:", + "This is not a partition column" + ) + + // BooleanType is not supported yet + val partitionColumnTypes = + Seq( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + DecimalType(10, 5), + DecimalType.SYSTEM_DEFAULT, + DateType, + SQLConf.get.timestampType, + StringType + ) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { + dir => + df.write + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name).cast(f.dataType)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { + dir => + df.write + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name).cast(f.dataType)) + checkAnswer( + spark.read + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .load(dir.toString) + .select(fields: _*), + row) + } + } + } + } + + test("gluten: Various inferred partition value types") { + Seq(TimestampTypes.TIMESTAMP_NTZ).foreach { + tsType => + withSQLConf(SQLConf.TIMESTAMP_TYPE.key -> tsType.toString) { + val ts = if (tsType == TimestampTypes.TIMESTAMP_LTZ) { + Timestamp.valueOf("1990-02-24 12:00:30") + } else { + LocalDateTime.parse("1990-02-24T12:00:30") + } + val row = + Row( + Long.MaxValue, + 4.5, + new java.math.BigDecimal(new BigInteger("1" * 20)), + java.sql.Date.valueOf("2015-05-23"), + ts, + "This is a string, /[]?=:", + "This is not a partition column" + ) + + val partitionColumnTypes = + Seq( + LongType, + DoubleType, + DecimalType(20, 0), + DateType, + SQLConf.get.timestampType, + StringType) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { + dir => + df.write + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { + dir => + df.write + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer( + spark.read + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .load(dir.toString) + .select(fields: _*), + row) + } + } + } + } +} + +class GlutenParquetV2PartitionDiscoverySuite + extends ParquetV2PartitionDiscoverySuite + with GlutenSQLTestsBaseTrait { + test("gluten: Various partition value types") { + Seq(TimestampTypes.TIMESTAMP_NTZ).foreach { + tsType => + withSQLConf(SQLConf.TIMESTAMP_TYPE.key -> tsType.toString) { + val ts = if (tsType == TimestampTypes.TIMESTAMP_LTZ) { + new Timestamp(0) + } else { + LocalDateTime.parse("1970-01-01T00:00:00") + } + val row = + Row( + 100.toByte, + 40000.toShort, + Int.MaxValue, + Long.MaxValue, + 1.5.toFloat, + 4.5, + new java.math.BigDecimal(new BigInteger("212500"), 5), + new java.math.BigDecimal("2.125"), + java.sql.Date.valueOf("2015-05-23"), + ts, + "This is a string, /[]?=:", + "This is not a partition column" + ) + + // BooleanType is not supported yet + val partitionColumnTypes = + Seq( + ByteType, + ShortType, + IntegerType, + LongType, + FloatType, + DoubleType, + DecimalType(10, 5), + DecimalType.SYSTEM_DEFAULT, + DateType, + SQLConf.get.timestampType, + StringType + ) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { + dir => + df.write + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name).cast(f.dataType)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { + dir => + df.write + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name).cast(f.dataType)) + checkAnswer( + spark.read + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .load(dir.toString) + .select(fields: _*), + row) + } + } + } + } + + test("gluten: Various inferred partition value types") { + Seq(TimestampTypes.TIMESTAMP_NTZ).foreach { + tsType => + withSQLConf(SQLConf.TIMESTAMP_TYPE.key -> tsType.toString) { + val ts = if (tsType == TimestampTypes.TIMESTAMP_LTZ) { + Timestamp.valueOf("1990-02-24 12:00:30") + } else { + LocalDateTime.parse("1990-02-24T12:00:30") + } + val row = + Row( + Long.MaxValue, + 4.5, + new java.math.BigDecimal(new BigInteger("1" * 20)), + java.sql.Date.valueOf("2015-05-23"), + ts, + "This is a string, /[]?=:", + "This is not a partition column" + ) + + val partitionColumnTypes = + Seq( + LongType, + DoubleType, + DecimalType(20, 0), + DateType, + SQLConf.get.timestampType, + StringType) + + val partitionColumns = partitionColumnTypes.zipWithIndex.map { + case (t, index) => StructField(s"p_$index", t) + } + + val schema = StructType(partitionColumns :+ StructField(s"i", StringType)) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { + dir => + df.write + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer(spark.read.load(dir.toString).select(fields: _*), row) + } + + withTempPath { + dir => + df.write + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .format("parquet") + .partitionBy(partitionColumns.map(_.name): _*) + .save(dir.toString) + val fields = schema.map(f => Column(f.name)) + checkAnswer( + spark.read + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .load(dir.toString) + .select(fields: _*), + row) + } + } + } + } +} diff --git a/gluten-core/src/main/java/io/glutenproject/vectorized/NativeSerializableObject.java b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala similarity index 64% rename from gluten-core/src/main/java/io/glutenproject/vectorized/NativeSerializableObject.java rename to gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala index 043d05b13962..f175910792bd 100644 --- a/gluten-core/src/main/java/io/glutenproject/vectorized/NativeSerializableObject.java +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetProtobufCompatibilitySuite.scala @@ -14,21 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.glutenproject.vectorized; +package org.apache.spark.sql.execution.datasources.parquet -/** ArrowBufBuilder. */ -public class NativeSerializableObject { - public int[] size; - public long[] memoryAddress; +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsBaseTrait} - /** - * Create an instance for NativeSerializableObject. - * - * @param memoryAddress native ArrowBuf data addr. - * @param size ArrowBuf size. - */ - public NativeSerializableObject(long[] memoryAddress, int[] size) { - this.memoryAddress = memoryAddress; - this.size = size; +class GlutenParquetProtobufCompatibilitySuite + extends ParquetProtobufCompatibilitySuite + with GlutenSQLTestsBaseTrait { + override protected def readResourceParquetFile(name: String): DataFrame = { + spark.read.parquet( + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name) } } diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.scala new file mode 100644 index 000000000000..96eeb5dcf8f6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetQuerySuite.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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf + +/** A test suite that tests various Parquet queries. */ +class GlutenParquetV1QuerySuite extends ParquetV1QuerySuite with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key + "_DISABLED" + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def withAllParquetReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")(code) + // Disabled: We don't yet support this case as of now + // test the vectorized reader + // withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true")(code) + } + + import testImplicits._ + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} + +class GlutenParquetV2QuerySuite extends ParquetV2QuerySuite with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key + "_DISABLED" + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def withAllParquetReaders(code: => Unit): Unit = { + // test the row-based reader + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false")(code) + // Disabled: We don't yet support this case as of now + // test the vectorized reader + // withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true")(code) + } + + import testImplicits._ + + test( + GlutenTestConstants.GLUTEN_TEST + + "SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + withAllParquetReaders { + withTempPath { + path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), Seq(null: String).toDF) + } + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala new file mode 100644 index 000000000000..647108424c1c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetRebaseDatetimeSuite.scala @@ -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. + */ +package org.apache.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy +import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.{CORRECTED, EXCEPTION, LEGACY} + +import java.sql.Date + +class GlutenParquetRebaseDatetimeV1Suite + extends ParquetRebaseDatetimeV1Suite + with GlutenSQLTestsBaseTrait { + + import testImplicits._ + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } + + private def inReadConfToOptions( + conf: String, + mode: LegacyBehaviorPolicy.Value): Map[String, String] = conf match { + case SQLConf.PARQUET_INT96_REBASE_MODE_IN_READ.key => + Map(ParquetOptions.INT96_REBASE_MODE -> mode.toString) + case _ => Map(ParquetOptions.DATETIME_REBASE_MODE -> mode.toString) + } + + private def runInMode(conf: String, modes: Seq[LegacyBehaviorPolicy.Value])( + f: Map[String, String] => Unit): Unit = { + modes.foreach(mode => withSQLConf(conf -> mode.toString)(f(Map.empty))) + withSQLConf(conf -> EXCEPTION.toString) { + modes.foreach(mode => f(inReadConfToOptions(conf, mode))) + } + } + + // gluten does not consider file metadata which indicates needs rebase or not + // it only supports write the parquet file as CORRECTED + test("gluten SPARK-31159: rebasing dates in write") { + val N = 8 + Seq(false, true).foreach { + dictionaryEncoding => + withTempPath { + dir => + val path = dir.getAbsolutePath + withSQLConf(SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key -> CORRECTED.toString) { + Seq + .tabulate(N)(_ => "1001-01-01") + .toDF("dateS") + .select($"dateS".cast("date").as("date")) + .repartition(1) + .write + .option("parquet.enable.dictionary", dictionaryEncoding) + .parquet(path) + } + + withAllParquetReaders { + // The file metadata indicates if it needs rebase or not, so we can always get the + // correct result regardless of the "rebase mode" config. + runInMode( + SQLConf.PARQUET_REBASE_MODE_IN_READ.key, + Seq(LEGACY, CORRECTED, EXCEPTION)) { + options => + checkAnswer( + spark.read.options(options).parquet(path), + Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-01")))) + } + + // Force to not rebase to prove the written datetime values are rebased + // and we will get wrong result if we don't rebase while reading. + // gluten not support this mode +// withSQLConf("spark.test.forceNoRebase" -> "true") { +// checkAnswer( +// spark.read.parquet(path), +// Seq.tabulate(N)(_ => Row(Date.valueOf("1001-01-07")))) +// } + } + } + } + } +} + +class GlutenParquetRebaseDatetimeV2Suite + extends ParquetRebaseDatetimeV2Suite + with GlutenSQLTestsBaseTrait { + + override protected def getResourceParquetFilePath(name: String): String = { + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + "/" + name + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala new file mode 100644 index 000000000000..7de5fd1fa293 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaPruningSuite.scala @@ -0,0 +1,44 @@ +/* + * 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.datasources.parquet +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.ExtendedSQLTest + +@ExtendedSQLTest +class GlutenParquetV1SchemaPruningSuite + extends ParquetV1SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + // disable column reader for nested type + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } +} + +@ExtendedSQLTest +class GlutenParquetV2SchemaPruningSuite + extends ParquetV2SchemaPruningSuite + with GlutenSQLTestsBaseTrait { + override protected val vectorizedReaderNestedEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_NESTED_COLUMN_ENABLED.key + "_DISABLED" + override def sparkConf: SparkConf = { + super.sparkConf.set("spark.memory.offHeap.size", "3g") + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala new file mode 100644 index 000000000000..dbf520e9109e --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetSchemaSuite.scala @@ -0,0 +1,25 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetSchemaInferenceSuite + extends ParquetSchemaInferenceSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenParquetSchemaSuite extends ParquetSchemaSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.scala new file mode 100644 index 000000000000..b1ea56b311e1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetThriftCompatibilitySuite.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.spark.sql.execution.datasources.parquet + +import org.apache.spark.sql.{GlutenSQLTestsBaseTrait, GlutenTestConstants, Row} + +class GlutenParquetThriftCompatibilitySuite + extends ParquetThriftCompatibilitySuite + with GlutenSQLTestsBaseTrait { + + private val parquetFilePath = + getWorkspaceFilePath("sql", "core", "src", "test", "resources").toString + + "/test-data/parquet-thrift-compat.snappy.parquet" + + test(GlutenTestConstants.GLUTEN_TEST + "Read Parquet file generated by parquet-thrift") { + logInfo(s"""Schema of the Parquet file written by parquet-thrift: + |${readParquetSchema(parquetFilePath.toString)} + """.stripMargin) + + checkAnswer( + spark.read.parquet(parquetFilePath.toString), + (0 until 10).map { + i => + val suits = Array("SPADES", "HEARTS", "DIAMONDS", "CLUBS") + + val nonNullablePrimitiveValues = Seq( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + // Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always + // treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume + // Thrift `STRING`s are encoded using UTF-8. + s"val_$i", + s"val_$i", + // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings + suits(i % 4) + ) + + val nullablePrimitiveValues = if (i % 3 == 0) { + Seq.fill(nonNullablePrimitiveValues.length)(null) + } else { + nonNullablePrimitiveValues + } + + val complexValues = Seq( + Seq.tabulate(3)(n => s"arr_${i + n}"), + // Thrift `SET`s are converted to Parquet `LIST`s + Seq(i), + Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap, + Seq + .tabulate(3) { + n => + (i + n) -> Seq.tabulate(3) { + m => Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + } + .toMap + ) + + Row(nonNullablePrimitiveValues ++ nullablePrimitiveValues ++ complexValues: _*) + } + ) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala new file mode 100644 index 000000000000..a0cf738e52a6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/GlutenParquetVectorizedSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenParquetVectorizedSuite extends ParquetVectorizedSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala new file mode 100644 index 000000000000..bb3b04388ee4 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/text/GlutenTextSuite.scala @@ -0,0 +1,281 @@ +/* + * 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.datasources.text + +import org.apache.spark.{SparkConf, TestUtils} +import org.apache.spark.sql.{AnalysisException, DataFrame, GlutenSQLTestsBaseTrait, QueryTest, Row, SaveMode} +import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.Utils + +import org.apache.hadoop.io.SequenceFile.CompressionType +import org.apache.hadoop.io.compress.GzipCodec + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + +abstract class GlutenTextSuite + extends QueryTest + with SharedSparkSession + with CommonFileDataSourceSuite { + import testImplicits._ + + override protected def dataSourceFormat = "text" + + test("reading text file") { + verifyFrame(spark.read.format("text").load(testFile)) + } + + test("SQLContext.read.text() API") { + verifyFrame(spark.read.text(testFile)) + } + + test("SPARK-12562 verify write.text() can handle column name beyond `value`") { + val df = spark.read.text(testFile).withColumnRenamed("value", "adwrasdf") + + val tempFile = Utils.createTempDir() + tempFile.delete() + df.write.text(tempFile.getCanonicalPath) + verifyFrame(spark.read.text(tempFile.getCanonicalPath)) + + Utils.deleteRecursively(tempFile) + } + + test("error handling for invalid schema") { + val tempFile = Utils.createTempDir() + tempFile.delete() + + val df = spark.range(2) + intercept[AnalysisException] { + df.write.text(tempFile.getCanonicalPath) + } + + intercept[AnalysisException] { + spark.range(2).select(df("id"), df("id") + 1).write.text(tempFile.getCanonicalPath) + } + } + + test("reading partitioned data using read.textFile()") { + val ds = spark.read.textFile(textPartitioned) + val data = ds.collect() + + assert(ds.schema == new StructType().add("value", StringType)) + assert(data.length == 2) + } + + test("support for partitioned reading using read.text()") { + val df = spark.read.text(textPartitioned) + val data = df.filter("year = '2015'").select("value").collect() + + assert(data(0) == Row("2015-test")) + assert(data.length == 1) + } + + test("SPARK-13503 Support to specify the option for compression codec for TEXT") { + val testDf = spark.read.text(testFile) + val extensionNameMap = Map("bzip2" -> ".bz2", "deflate" -> ".deflate", "gzip" -> ".gz") + extensionNameMap.foreach { + case (codecName, extension) => + val tempDir = Utils.createTempDir() + val tempDirPath = tempDir.getAbsolutePath + testDf.write.option("compression", codecName).mode(SaveMode.Overwrite).text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(_.getName.endsWith(s".txt$extension"))) + verifyFrame(spark.read.text(tempDirPath)) + } + + val errMsg = intercept[IllegalArgumentException] { + val tempDirPath = Utils.createTempDir().getAbsolutePath + testDf.write.option("compression", "illegal").mode(SaveMode.Overwrite).text(tempDirPath) + } + assert( + errMsg.getMessage.contains("Codec [illegal] is not available. " + + "Known codecs are")) + } + + test("SPARK-13543 Write the output as uncompressed via option()") { + val extraOptions = Map[String, String]( + "mapreduce.output.fileoutputformat.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mapreduce.map.output.compress" -> "true", + "mapreduce.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mapreduce.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("compression", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + test("case insensitive option") { + val extraOptions = Map[String, String]( + "mApReDuCe.output.fileoutputformat.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.type" -> CompressionType.BLOCK.toString, + "mApReDuCe.map.output.compress" -> "true", + "mApReDuCe.output.fileoutputformat.compress.codec" -> classOf[GzipCodec].getName, + "mApReDuCe.map.output.compress.codec" -> classOf[GzipCodec].getName + ) + withTempDir { + dir => + val testDf = spark.read.text(testFile) + val tempDirPath = dir.getAbsolutePath + testDf.write + .option("CoMpReSsIoN", "none") + .options(extraOptions) + .mode(SaveMode.Overwrite) + .text(tempDirPath) + val compressedFiles = new File(tempDirPath).listFiles() + assert(compressedFiles.exists(!_.getName.endsWith(".txt.gz"))) + verifyFrame(spark.read.options(extraOptions).text(tempDirPath)) + } + } + + test("SPARK-14343: select partitioning column") { + withTempPath { + dir => + val path = dir.getCanonicalPath + val ds1 = spark.range(1).selectExpr("CONCAT('val_', id)") + ds1.write.text(s"$path/part=a") + ds1.write.text(s"$path/part=b") + + checkAnswer( + spark.read.format("text").load(path).select($"part"), + Row("a") :: Row("b") :: Nil) + } + } + + test("SPARK-15654: should not split gz files") { + withTempDir { + dir => + val path = dir.getCanonicalPath + val df1 = spark.range(0, 1000).selectExpr("CAST(id AS STRING) AS s") + df1.write.option("compression", "gzip").mode("overwrite").text(path) + + val expected = df1.collect() + Seq(10, 100, 1000).foreach { + bytes => + withSQLConf(SQLConf.FILES_MAX_PARTITION_BYTES.key -> bytes.toString) { + val df2 = spark.read.format("text").load(path) + checkAnswer(df2, expected) + } + } + } + } + + def testLineSeparator(lineSep: String): Unit = { + test(s"SPARK-23577: Support line separator - lineSep: '$lineSep'") { + // Read + val values = Seq("a", "b", "\nc") + val data = values.mkString(lineSep) + val dataWithTrailingLineSep = s"$data$lineSep" + Seq(data, dataWithTrailingLineSep).foreach { + lines => + withTempPath { + path => + Files.write(path.toPath, lines.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, Seq("a", "b", "\nc").toDF()) + } + } + + // Write + withTempPath { + path => + values.toDF().coalesce(1).write.option("lineSep", lineSep).text(path.getAbsolutePath) + val partFile = + TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), StandardCharsets.UTF_8) + assert(readBack === s"a${lineSep}b$lineSep\nc$lineSep") + } + + // Roundtrip + withTempPath { + path => + val df = values.toDF() + df.write.option("lineSep", lineSep).text(path.getAbsolutePath) + val readBack = spark.read.option("lineSep", lineSep).text(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + Seq("|", "^", "::", "!!!@3", 0x1e.toChar.toString, "아").foreach { + lineSep => testLineSeparator(lineSep) + } + // scalastyle:on nonascii + + // Rewrite for file locating. + private def testFile: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-suite.txt" + } + + // Added for file locating. + private def textPartitioned: String = { + getWorkspaceFilePath( + "sql", + "core", + "src", + "test", + "resources").toString + "/test-data/text-partitioned" + } + + /** Verifies data and schema. */ + private def verifyFrame(df: DataFrame): Unit = { + // schema + assert(df.schema == new StructType().add("value", StringType)) + + // verify content + val data = df.collect() + assert(data(0) == Row("This is a test file for the text data source")) + assert(data(1) == Row("1+1")) + // scalastyle:off nonascii + assert(data(2) == Row("数据砖头")) + // scalastyle:on nonascii + assert(data(3) == Row("\"doh\"")) + assert(data.length == 4) + } +} + +class GlutenTextV1Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "text") +} + +class GlutenTextV2Suite extends GlutenTextSuite with GlutenSQLTestsBaseTrait { + override def sparkConf: SparkConf = + super.sparkConf + .set(SQLConf.USE_V1_SOURCE_LIST, "") +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala new file mode 100644 index 000000000000..f6d7db3849e9 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenDataSourceV2StrategySuite.scala @@ -0,0 +1,23 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDataSourceV2StrategySuite + extends DataSourceV2StrategySuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala new file mode 100644 index 000000000000..bc6fcc3c0e9b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenFileTableSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenFileTableSuite extends FileTableSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala new file mode 100644 index 000000000000..e2d8186f6874 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/datasources/v2/GlutenV2PredicateSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenV2PredicateSuite extends V2PredicateSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.scala new file mode 100644 index 000000000000..24c66b51a536 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/exchange/GlutenEnsureRequirementsSuite.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.spark.sql.execution.exchange + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait +import org.apache.spark.sql.GlutenTestConstants.GLUTEN_TEST +import org.apache.spark.sql.internal.SQLConf + +class GlutenEnsureRequirementsSuite extends EnsureRequirementsSuite with GlutenSQLTestsBaseTrait { + + test( + GLUTEN_TEST + + "SPARK-35675: EnsureRequirements remove shuffle should respect PartitioningCollection") { + import testImplicits._ + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1", + SQLConf.SHUFFLE_PARTITIONS.key -> "5", + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df1 = Seq((1, 2)).toDF("c1", "c2") + val df2 = Seq((1, 3)).toDF("c3", "c4") + val res = df1.join(df2, $"c1" === $"c3").repartition($"c1") + assert(res.queryExecution.executedPlan.collect { case s: ShuffleExchangeLike => s }.size == 2) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala new file mode 100644 index 000000000000..b05074c8d14c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenBroadcastJoinSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.joins + +import io.glutenproject.GlutenConfig +import io.glutenproject.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.sql.{GlutenTestsCommonTrait, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.internal.SQLConf + +/** + * This test needs setting for spark test home (its source code), e.g., appending the following + * setting for `mvn test`: -DargLine="-Dspark.test.home=/home/sparkuser/spark/". + * + * In addition, you also need build spark source code before running this test, e.g., with + * `./build/mvn -DskipTests clean package`. + */ +class GlutenBroadcastJoinSuite extends BroadcastJoinSuite with GlutenTestsCommonTrait { + + /** + * Create a new [[SparkSession]] running in local-cluster mode with unsafe and codegen enabled. + */ + override def beforeAll(): Unit = { + super.beforeAll() + val sparkBuilder = SparkSession + .builder() + .master("local-cluster[2,1,1024]") + .appName("Gluten-UT") + .master(s"local[2]") + .config(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "io.glutenproject.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .config("spark.sql.warehouse.dir", warehouse) + // Avoid static evaluation for literal input by spark catalyst. + .config( + "spark.sql.optimizer.excludedRules", + ConstantFolding.ruleName + "," + + NullPropagation.ruleName) + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config("spark.gluten.sql.columnar.backend.ch.use.v2", "false") + .config("spark.gluten.sql.enable.native.validation", "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala new file mode 100644 index 000000000000..309af61a43ae --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenExistenceJoinSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.joins + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExistenceJoinSuite extends ExistenceJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala new file mode 100644 index 000000000000..745f550ae35a --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenInnerJoinSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.joins + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenInnerJoinSuite extends InnerJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala new file mode 100644 index 000000000000..c915c73695b6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/execution/joins/GlutenOuterJoinSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.joins + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenOuterJoinSuite extends OuterJoinSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala new file mode 100644 index 000000000000..fe37da206a56 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/CustomerColumnarPreRules.scala @@ -0,0 +1,44 @@ +/* + * 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.extension + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} + +case class CustomerColumnarPreRules(session: SparkSession) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = plan.transformDown { + case fileSourceScan: FileSourceScanExec => + val transformer = new TestFileSourceScanExecTransformer( + fileSourceScan.relation, + fileSourceScan.output, + fileSourceScan.requiredSchema, + fileSourceScan.partitionFilters, + fileSourceScan.optionalBucketSet, + fileSourceScan.optionalNumCoalescedBuckets, + fileSourceScan.dataFilters, + fileSourceScan.tableIdentifier, + fileSourceScan.disableBucketedScan + ) + if (transformer.doValidate().isValid) { + transformer + } else { + plan + } + } +} 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 new file mode 100644 index 000000000000..f55a5c3c41a2 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenCustomerExtensionSuite.scala @@ -0,0 +1,49 @@ +/* + * 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.extension + +import org.apache.spark.SparkConf +import org.apache.spark.sql.GlutenSQLTestsTrait + +class GlutenCustomerExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set("spark.sql.adaptive.enabled", "false") + .set( + "spark.gluten.sql.columnar.extended.columnar.pre.rules", + "org.apache.spark.sql" + + ".extension.CustomerColumnarPreRules") + .set("spark.gluten.sql.columnar.extended.columnar.post.rules", "") + } + + test("test customer column rules") { + withSQLConf(("spark.gluten.enabled", "false")) { + sql("create table my_parquet(id int) using parquet") + sql("insert into my_parquet values (1)") + sql("insert into my_parquet values (2)") + } + withSQLConf(("spark.gluten.sql.columnar.filescan", "false")) { + val df = sql("select * from my_parquet") + val testFileSourceScanExecTransformer = df.queryExecution.executedPlan.collect { + case f: TestFileSourceScanExecTransformer => f + } + assert(!testFileSourceScanExecTransformer.isEmpty) + assert(testFileSourceScanExecTransformer(0).nodeNamePrefix.equals("TestNativeFile")) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala new file mode 100644 index 000000000000..847f066bf4d6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/GlutenSessionExtensionSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.extension + +import io.glutenproject.extension.{ColumnarOverrideRules, FallbackBroadcastExchange, JoinSelectionOverrides} +import io.glutenproject.extension.columnar.{FallbackMultiCodegens, FallbackOnANSIMode} + +import org.apache.spark.SparkConf +import org.apache.spark.sql._ +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS + +class GlutenSessionExtensionSuite extends GlutenSQLTestsTrait { + + override def sparkConf: SparkConf = { + super.sparkConf + .set(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) + } + + test("test gluten extensions") { + assert( + spark.sessionState.adaptiveRulesHolder.queryStagePrepRules + .contains(FallbackOnANSIMode(spark))) + assert( + spark.sessionState.adaptiveRulesHolder.queryStagePrepRules + .contains(FallbackMultiCodegens(spark))) + assert( + spark.sessionState.adaptiveRulesHolder.queryStagePrepRules + .contains(FallbackBroadcastExchange(spark))) + assert(spark.sessionState.columnarRules.contains(ColumnarOverrideRules(spark))) + assert(spark.sessionState.planner.strategies.contains(JoinSelectionOverrides(spark))) + + assert(spark.sessionState.planner.strategies.contains(MySparkStrategy(spark))) + assert(spark.sessionState.analyzer.extendedResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.postHocResolutionRules.contains(MyRule(spark))) + assert(spark.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(spark))) + assert(spark.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(spark))) + assert(spark.sessionState.sqlParser.isInstanceOf[MyParser]) + assert( + spark.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1) + .isDefined) + assert( + spark.sessionState.columnarRules.contains( + MyColumnarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + } +} 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 new file mode 100644 index 000000000000..36fe19bd7039 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/extension/TestFileSourceScanExecTransformer.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.spark.sql.extension + +import io.glutenproject.backendsapi.BackendsApiManager +import io.glutenproject.execution.FileSourceScanExecTransformer + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.connector.read.InputPartition +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet + +/** Test for customer column rules */ +class TestFileSourceScanExecTransformer( + @transient relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + dataFilters: Seq[Expression], + tableIdentifier: Option[TableIdentifier], + disableBucketedScan: Boolean = false) + extends FileSourceScanExecTransformer( + relation, + output, + requiredSchema, + partitionFilters, + optionalBucketSet, + optionalNumCoalescedBuckets, + dataFilters, + tableIdentifier, + disableBucketedScan) { + override def getPartitions: Seq[InputPartition] = + BackendsApiManager.getTransformerApiInstance.genInputPartitionSeq( + relation, + selectedPartitions, + output, + bucketedScan, + optionalBucketSet, + optionalNumCoalescedBuckets, + disableBucketedScan) + + override val nodeNamePrefix: String = "TestNativeFile" +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala new file mode 100644 index 000000000000..44e848a31e7c --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/gluten/GlutenFallbackSuite.scala @@ -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. + */ +package org.apache.spark.sql.gluten + +import io.glutenproject.{GlutenConfig, VERSION} + +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.GlutenSQLTestsTrait +import org.apache.spark.sql.execution.ui.{GlutenSQLAppStatusStore, SparkListenerSQLExecutionStart} +import org.apache.spark.status.ElementTrackingStore + +class GlutenFallbackSuite extends GlutenSQLTestsTrait { + + ignore("test fallback logging") { + val testAppender = new LogAppender("fallback reason") + withLogAppender(testAppender) { + withSQLConf( + GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false", + GlutenConfig.VALIDATION_LOG_LEVEL.key -> "error") { + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + sql("SELECT * FROM t").collect() + } + } + assert( + testAppender.loggingEvents.exists( + _.getMessage.getFormattedMessage.contains( + "Validation failed for plan: Scan parquet default.t, " + + "due to: columnar FileScan is not enabled in FileSourceScanExec"))) + } + } + + ignore("test fallback event") { + val kvStore = spark.sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] + val glutenStore = new GlutenSQLAppStatusStore(kvStore) + assert(glutenStore.buildInfo().info.find(_._1 == "Gluten Version").exists(_._2 == VERSION)) + + def runExecution(sqlString: String): Long = { + var id = 0L + val listener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerSQLExecutionStart => id = e.executionId + case _ => + } + } + } + spark.sparkContext.addSparkListener(listener) + try { + sql(sqlString).collect() + spark.sparkContext.listenerBus.waitUntilEmpty() + } finally { + spark.sparkContext.removeSparkListener(listener) + } + id + } + + withTable("t") { + spark.range(10).write.format("parquet").saveAsTable("t") + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 2) + assert(execution.get.numFallbackNodes == 0) + assert(execution.get.fallbackNodeToReason.isEmpty) + + withSQLConf(GlutenConfig.COLUMNAR_FILESCAN_ENABLED.key -> "false") { + val id = runExecution("SELECT * FROM t") + val execution = glutenStore.execution(id) + assert(execution.isDefined) + assert(execution.get.numGlutenNodes == 0) + assert(execution.get.numFallbackNodes == 2) + val fallbackReason = execution.get.fallbackNodeToReason.head + assert(fallbackReason._1.contains("Scan parquet default.t")) + assert(fallbackReason._2.contains("columnar FileScan is not enabled in FileSourceScanExec")) + } + } + + withTable("t1", "t2") { + spark.range(10).write.format("parquet").saveAsTable("t1") + spark.range(10).write.format("parquet").saveAsTable("t2") + + val id = runExecution("SELECT * FROM t1 JOIN t2") + val execution = glutenStore.execution(id) + // broadcast exchange and broadcast nested loop join + assert(execution.get.numFallbackNodes == 2) + assert( + execution.get.fallbackNodeToReason.head._2 + .contains("Gluten does not touch it or does not support it")) + } + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala new file mode 100644 index 000000000000..d82e84d50d96 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/hive/execution/GlutenHiveSQLQuerySuite.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution + +import io.glutenproject.execution.TransformSupport + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI.UI_ENABLED +import org.apache.spark.sql.{DataFrame, GlutenSQLTestsTrait, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, ShuffleQueryStageExec} +import org.apache.spark.sql.hive.{HiveTableScanExecTransformer, HiveUtils} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} + +import scala.reflect.ClassTag + +class GlutenHiveSQLQuerySuite extends GlutenSQLTestsTrait { + private var _spark: SparkSession = null + + override def beforeAll(): Unit = { + prepareWorkDir() + if (_spark == null) { + _spark = SparkSession.builder().config(sparkConf).enableHiveSupport().getOrCreate() + } + + _spark.sparkContext.setLogLevel("info") + } + + override protected def spark: SparkSession = _spark + + override def afterAll(): Unit = { + try { + super.afterAll() + if (_spark != null) { + try { + _spark.sessionState.catalog.reset() + } finally { + _spark.stop() + _spark = null + } + } + } finally { + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + doThreadPostAudit() + } + } + + protected def defaultSparkConf: SparkConf = { + val conf = new SparkConf() + .set("spark.master", "local[1]") + .set("spark.sql.test", "") + .set("spark.sql.testkey", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.CODEGEN_FACTORY_MODE.key, CodegenObjectFactoryMode.CODEGEN_ONLY.toString) + .set( + HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, + "org.apache.spark.sql.hive.execution.PairSerDe") + // SPARK-8910 + .set(UI_ENABLED, false) + .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) + // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes + // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. + .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") + // Disable ConvertToLocalRelation for better test coverage. Test cases built on + // LocalRelation will exercise the optimization rules better by disabling it as + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + + conf.set( + StaticSQLConf.WAREHOUSE_PATH, + conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) + } + + /** + * 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 plan => + getChildrenPlan(Seq(plan)) + } + } + + def checkOperatorMatch[T <: TransformSupport](df: DataFrame)(implicit tag: ClassTag[T]): Unit = { + val executedPlan = getExecutedPlan(df) + assert(executedPlan.exists(plan => plan.getClass == tag.runtimeClass)) + } + + override def sparkConf: SparkConf = { + defaultSparkConf + .set("spark.plugins", "io.glutenproject.GlutenPlugin") + .set("spark.default.parallelism", "1") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "1024MB") + } + + test("hive orc scan") { + withSQLConf("spark.sql.hive.convertMetastoreOrc" -> "false") { + sql("DROP TABLE IF EXISTS test_orc") + sql( + "CREATE TABLE test_orc (name STRING, favorite_color STRING)" + + " USING hive OPTIONS(fileFormat 'orc')") + sql("INSERT INTO test_orc VALUES('test_1', 'red')"); + val df = spark.sql("select * from test_orc") + checkAnswer(df, Seq(Row("test_1", "red"))) + checkOperatorMatch[HiveTableScanExecTransformer](df) + } + spark.sessionState.catalog.dropTable( + TableIdentifier("test_orc"), + ignoreIfNotExists = true, + purge = false) + } +} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala new file mode 100644 index 000000000000..9a9f06e02c5d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedReadSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.sources + +import org.apache.spark.sql._ + +class GlutenBucketedReadWithoutHiveSupportSuite + extends BucketedReadWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala new file mode 100644 index 000000000000..e5dd2de8b8bd --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenBucketedWriteSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.sources +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenBucketedWriteWithoutHiveSupportSuite + extends BucketedWriteWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala new file mode 100644 index 000000000000..7f31d62f74be --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenCreateTableAsSelectSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenCreateTableAsSelectSuite + extends CreateTableAsSelectSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala new file mode 100644 index 000000000000..03775cab3914 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDDLSourceLoadSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +// please note that the META-INF/services had to be modified for the test directory for this to work +class GlutenDDLSourceLoadSuite extends DDLSourceLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.scala new file mode 100644 index 000000000000..fd77663985bc --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenDisableUnnecessaryBucketedScanSuite.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.sql.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuite + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuite + with GlutenSQLTestsBaseTrait {} + +class GlutenDisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + extends DisableUnnecessaryBucketedScanWithoutHiveSupportSuiteAE + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala new file mode 100644 index 000000000000..84ba336099a1 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenExternalCommandRunnerSuite.scala @@ -0,0 +1,23 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenExternalCommandRunnerSuite + extends ExternalCommandRunnerSuite + with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala new file mode 100644 index 000000000000..d751f20ae3f6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFilteredScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql._ + +class GlutenFilteredScanSuite extends FilteredScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala new file mode 100644 index 000000000000..ad91b92aae20 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenFiltersSuite.scala @@ -0,0 +1,22 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +/** Unit test suites for data source filters. */ +class GlutenFiltersSuite extends FiltersSuite with GlutenSQLTestsBaseTrait {} 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 new file mode 100644 index 000000000000..165d51731302 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenInsertSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql._ + +class GlutenInsertSuite extends InsertSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala new file mode 100644 index 000000000000..26c847ff2323 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPartitionedWriteSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPartitionedWriteSuite extends PartitionedWriteSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala new file mode 100644 index 000000000000..94171f44cecc --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPathOptionSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenPathOptionSuite extends PathOptionSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala new file mode 100644 index 000000000000..920d4f3af647 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenPrunedScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql._ + +class GlutenPrunedScanSuite extends PrunedScanSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala new file mode 100644 index 000000000000..ddd06bb3fd89 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenResolvedDataSourceSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenResolvedDataSourceSuite extends ResolvedDataSourceSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala new file mode 100644 index 000000000000..5ae0204b835d --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenSaveLoadSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql.GlutenSQLTestsBaseTrait + +class GlutenSaveLoadSuite extends SaveLoadSuite with GlutenSQLTestsBaseTrait {} diff --git a/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala new file mode 100644 index 000000000000..ebd17781ff2b --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/sources/GlutenTableScanSuite.scala @@ -0,0 +1,21 @@ +/* + * 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.sources + +import org.apache.spark.sql._ + +class GlutenTableScanSuite extends TableScanSuite with GlutenSQLTestsBaseTrait {} 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 new file mode 100644 index 000000000000..bec5be2fc3a6 --- /dev/null +++ b/gluten-ut/spark34/src/test/scala/org/apache/spark/sql/statistics/SparkFunctionStatistics.scala @@ -0,0 +1,220 @@ +/* + * 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.statistics + +import io.glutenproject.GlutenConfig +import io.glutenproject.extension.GlutenPlan +import io.glutenproject.utils.{BackendTestUtils, SystemParameters} + +import org.apache.spark.sql.{GlutenTestConstants, QueryTest, SparkSession} +import org.apache.spark.sql.catalyst.optimizer.{ConstantFolding, ConvertToLocalRelation, NullPropagation} +import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.internal.SQLConf + +import scala.util.control.Breaks.{break, breakable} + +/** + * TODO: There are some false positive & false negative cases for some functions. For such + * situation, we need to use a suitable test sql to do the check. + */ +class SparkFunctionStatistics extends QueryTest { + + var spark: SparkSession = null + + protected def initializeSession(): Unit = { + if (spark == null) { + val sparkBuilder = SparkSession + .builder() + .appName("Gluten-UT") + .master(s"local[2]") + // Avoid static evaluation for literal input by spark catalyst. + .config( + SQLConf.OPTIMIZER_EXCLUDED_RULES.key, + ConvertToLocalRelation.ruleName + + "," + ConstantFolding.ruleName + "," + NullPropagation.ruleName) + .config("spark.driver.memory", "1G") + .config("spark.sql.adaptive.enabled", "true") + .config("spark.sql.shuffle.partitions", "1") + .config("spark.sql.files.maxPartitionBytes", "134217728") + .config("spark.memory.offHeap.enabled", "true") + .config("spark.memory.offHeap.size", "1024MB") + .config("spark.plugins", "io.glutenproject.GlutenPlugin") + .config("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + // Avoid the code size overflow error in Spark code generation. + .config("spark.sql.codegen.wholeStage", "false") + + spark = if (BackendTestUtils.isCHBackendLoaded()) { + sparkBuilder + .config("spark.io.compression.codec", "LZ4") + .config("spark.gluten.sql.columnar.backend.ch.worker.id", "1") + .config("spark.gluten.sql.columnar.backend.ch.use.v2", "false") + .config("spark.gluten.sql.enable.native.validation", "false") + .config("spark.sql.files.openCostInBytes", "134217728") + .config(GlutenConfig.GLUTEN_LIB_PATH, SystemParameters.getClickHouseLibPath) + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } else { + sparkBuilder + .config("spark.unsafe.exceptionOnMemoryLeak", "true") + .getOrCreate() + } + } + } + + def extractQuery(examples: String): Seq[String] = { + examples + .split("\n") + .map(_.trim) + .filter(!_.isEmpty) + .filter(_.startsWith("> SELECT")) + .map(_.replace("> SELECT", "SELECT")) + } + + test(GlutenTestConstants.GLUTEN_TEST + "Run spark function statistics: ") { + initializeSession + val functionRegistry = spark.sessionState.functionRegistry + val sparkBuiltInFunctions = functionRegistry.listFunction() + // According to expressionsForTimestampNTZSupport in FunctionRegistry.scala, + // these functions are registered only for testing, not available for end users. + // Other functions like current_database is NOT necessarily offloaded to native. + val ignoreFunctions = Seq( + "get_fake_app_name", + "current_catalog", + "current_database", + "spark_partition_id", + "current_user", + "current_timezone") + val supportedFunctions = new java.util.ArrayList[String]() + val unsupportedFunctions = new java.util.ArrayList[String]() + val needInspectFunctions = new java.util.ArrayList[String]() + + for (func <- sparkBuiltInFunctions) { + val exprInfo = functionRegistry.lookupFunction(func).get + if (!ignoreFunctions.contains(exprInfo.getName)) { + val examples = extractQuery(exprInfo.getExamples) + if (examples.isEmpty) { + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("## Not found examples for " + exprInfo.getName) + // scalastyle:on println + } + var isSupported: Boolean = true + breakable { + for (example <- examples) { + var executedPlan: SparkPlan = null + try { + executedPlan = spark.sql(example).queryExecution.executedPlan + } catch { + case t: Throwable => + needInspectFunctions.add(exprInfo.getName) + // scalastyle:off println + println("-- Need inspect " + exprInfo.getName) + println(exprInfo.getExamples) + // scalastyle:on println + break + } + val hasFallbackProject = executedPlan.find(_.isInstanceOf[ProjectExec]).isDefined + if (hasFallbackProject) { + isSupported = false + break + } + val hasGlutenPlan = executedPlan.find(_.isInstanceOf[GlutenPlan]).isDefined + if (!hasGlutenPlan) { + isSupported = false + break + } + break + } + } + if (isSupported && !needInspectFunctions.contains(exprInfo.getName)) { + supportedFunctions.add(exprInfo.getName) + } else if (!isSupported) { + unsupportedFunctions.add(exprInfo.getName) + } + } + } + // scalastyle:off println + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions: " + supportedFunctions.size()) + println("Unsupported functions: " + unsupportedFunctions.size()) + println("Need inspect functions: " + needInspectFunctions.size()) + // scalastyle:on println + // For correction. + val supportedCastAliasFunctions = Seq( + "boolean", + "tinyint", + "smallint", + "int", + "bigint", + "float", + "double", + "decimal", + "date", + "binary", + "string") + for (func <- supportedCastAliasFunctions) { + if (needInspectFunctions.contains(func)) { + needInspectFunctions.remove(func) + supportedFunctions.add(func) + } + } + + // For wrongly recognized unsupported case. + Seq("%", "ceil", "floor", "first", "first_value", "last", "last_value", "hash", "mod").foreach( + name => { + if (unsupportedFunctions.remove(name)) { + supportedFunctions.add(name) + } + }) + // For wrongly recognized supported case. + Seq( + "array_contains", + "map_keys", + "get_json_object", + "element_at", + "map_from_arrays", + "contains", + "startswith", + "endswith", + "map_contains_key", + "map_values", + "try_element_at", + "struct", + "array", + "ilike", + "sec", + "csc" + ).foreach( + name => { + if (supportedFunctions.remove(name)) { + unsupportedFunctions.add(name) + } + }) + // Functions in needInspectFunctions were checked. + unsupportedFunctions.addAll(needInspectFunctions) + // scalastyle:off println + println("---------------") + println("Overall functions: " + (sparkBuiltInFunctions.size - ignoreFunctions.size)) + println("Supported functions corrected: " + supportedFunctions.size()) + println("Unsupported functions corrected: " + unsupportedFunctions.size()) + println("Support list:") + println(supportedFunctions) + println("Not support list:") + println(unsupportedFunctions) + // scalastyle:on println + } +} diff --git a/package/pom.xml b/package/pom.xml index ba5886cc6dfb..af65203dfc79 100644 --- a/package/pom.xml +++ b/package/pom.xml @@ -50,7 +50,7 @@ org.apache.spark - spark-hive-thriftserver_${scala.binary.version} + spark-hive_${scala.binary.version} provided @@ -198,6 +198,9 @@ org.apache.spark.unused.UnusedStubClass javax.jdo.* + + javax.transaction.* + javax.xml.* org.apache.commons.logging.* @@ -250,6 +253,8 @@ org.apache.spark.sql.execution.datasources.EmptyDirectoryDataWriter$ org.apache.spark.sql.execution.datasources.WriterBucketSpec org.apache.spark.sql.execution.datasources.WriterBucketSpec$ + org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand + org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$ diff --git a/pom.xml b/pom.xml index 9a2b836dbf86..3273854b98fd 100644 --- a/pom.xml +++ b/pom.xml @@ -30,34 +30,27 @@ - shims gluten-core + gluten-ui package + shims substrait/substrait-spark - gluten-ui 2.9.3 - 3.2.2 - 2.12.15 - 3.2 - 3.3.1 - 2.12.15 - 3.3 - 2.0.1 - 2.2.0 - ${delta20.version} + 2.0.1 20 2.12 - ${spark32.scala} - ${spark32.version} + 2.12.15 3 + 3.2 + 3.4.1 + spark-sql-columnar-shims-spark32 0.3.0-incubating - ${spark32bundle.version} 12.0.0 arrow-memory-unsafe - ${hadoop.version} + 2.7.4 UTF-8 UTF-8 spark-sql-columnar @@ -68,8 +61,7 @@ /usr/local/clickhouse/lib/libch.so /data/tpcds-data-sf1 - 2.13.3 - ${fasterxml.spark33.version} + 2.13.5 4.13.1 0.5.0 @@ -104,8 +96,6 @@ 3.2.2 1.0.0 2.2.0 - spark-sql-columnar-shims-spark32 - spark-sql-columnar-shims-spark33 @@ -115,34 +105,35 @@ true - ${spark32.scala} - 3 - ${spark32.version} - ${delta20.version} + 3.2 + spark-sql-columnar-shims-spark32 + 3.2.2 + 2.0.1 20 - ${spark32bundle.version} - ${spark32.shim.version} spark-3.3 - ${spark33.scala} - 3 - ${spark33.version} - ${delta22.version} + 3.3 + spark-sql-columnar-shims-spark33 + 3.3.1 + 2.2.0 + 22 + + + + spark-3.4 + + 3.4 + spark-sql-columnar-shims-spark34 + 3.4.1 + 2.2.0 22 - ${spark33bundle.version} - ${spark33.shim.version} hadoop-2.7.4 - - - !hadoop.version - - 2.7.4 @@ -203,6 +194,15 @@ gluten-celeborn + + delta + + false + + + gluten-delta + + backends-velox @@ -271,7 +271,7 @@ org.apache.spark - spark-hive-thriftserver_${scala.binary.version} + spark-hive_${scala.binary.version} ${spark.version} provided diff --git a/shims/common/pom.xml b/shims/common/pom.xml index 88c4feaac14f..70ec2a1c2a65 100644 --- a/shims/common/pom.xml +++ b/shims/common/pom.xml @@ -32,13 +32,11 @@ org.apache.spark spark-sql_${scala.binary.version} - ${spark.version} provided org.apache.spark spark-hive_${scala.binary.version} - ${spark.version} provided diff --git a/shims/common/src/main/scala/io/glutenproject/GlutenConfig.scala b/shims/common/src/main/scala/io/glutenproject/GlutenConfig.scala index 1751d2672bbb..0922ac4088e2 100644 --- a/shims/common/src/main/scala/io/glutenproject/GlutenConfig.scala +++ b/shims/common/src/main/scala/io/glutenproject/GlutenConfig.scala @@ -61,6 +61,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableColumnarWindow: Boolean = conf.getConf(COLUMNAR_WINDOW_ENABLED) + def veloxColumnarWindowType: String = conf.getConfString(COLUMNAR_VELOX_WINDOW_TYPE.key) + def enableColumnarShuffledHashJoin: Boolean = conf.getConf(COLUMNAR_SHUFFLED_HASH_JOIN_ENABLED) def enableNativeColumnarToRow: Boolean = conf.getConf(COLUMNAR_COLUMNAR_TO_ROW_ENABLED) @@ -83,6 +85,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def columnarTableCacheEnabled: Boolean = conf.getConf(COLUMNAR_TABLE_CACHE_ENABLED) + def enableDateTimestampComparison: Boolean = conf.getConf(ENABLE_DATE_TIMESTAMP_COMPARISON) + // whether to use ColumnarShuffleManager def isUseColumnarShuffleManager: Boolean = conf @@ -101,8 +105,6 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableOneRowRelationColumnar: Boolean = conf.getConf(COLUMNAR_ONE_ROW_RELATION_ENABLED) - def enableColumnarIterator: Boolean = conf.getConf(COLUMNAR_ITERATOR_ENABLED) - def physicalJoinOptimizationThrottle: Integer = conf.getConf(COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE) @@ -140,8 +142,8 @@ class GlutenConfig(conf: SQLConf) extends Logging { def columnarShuffleEnableIaa: Boolean = columnarShuffleCodecBackend.contains(GlutenConfig.GLUTEN_IAA_BACKEND_NAME) - def columnarShuffleBufferCompressThreshold: Int = - conf.getConf(COLUMNAR_SHUFFLE_BUFFER_COMPRESS_THRESHOLD) + def columnarShuffleCompressionThreshold: Int = + conf.getConf(COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD) def maxBatchSize: Int = conf.getConf(COLUMNAR_MAX_BATCH_SIZE) @@ -228,6 +230,13 @@ class GlutenConfig(conf: SQLConf) extends Logging { def veloxSpillFileSystem: String = conf.getConf(COLUMNAR_VELOX_SPILL_FILE_SYSTEM) + def veloxBloomFilterExpectedNumItems: Long = + conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS) + + def veloxBloomFilterNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_NUM_BITS) + + def veloxBloomFilterMaxNumBits: Long = conf.getConf(COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS) + def chColumnarShufflePreferSpill: Boolean = conf.getConf(COLUMNAR_CH_SHUFFLE_PREFER_SPILL_ENABLED) def chColumnarShuffleSpillThreshold: Long = conf.getConf(COLUMNAR_CH_SHUFFLE_SPILL_THRESHOLD) @@ -258,6 +267,9 @@ class GlutenConfig(conf: SQLConf) extends Logging { def enableVeloxUserExceptionStacktrace: Boolean = conf.getConf(COLUMNAR_VELOX_ENABLE_USER_EXCEPTION_STACKTRACE) + def memoryUseHugePages: Boolean = + conf.getConf(COLUMNAR_VELOX_MEMORY_USE_HUGE_PAGES) + def debug: Boolean = conf.getConf(DEBUG_LEVEL_ENABLED) def taskStageId: Int = conf.getConf(BENCHMARK_TASK_STAGEID) def taskPartitionId: Int = conf.getConf(BENCHMARK_TASK_PARTITIONID) @@ -298,6 +310,7 @@ object GlutenConfig { val SPARK_SQL_PARQUET_COMPRESSION_CODEC: String = "spark.sql.parquet.compression.codec" val PARQUET_BLOCK_SIZE: String = "parquet.block.size" val PARQUET_BLOCK_ROWS: String = "parquet.block.rows" + val PARQUET_GZIP_WINDOW_SIZE: String = "parquet.gzip.windowSize" // Hadoop config val HADOOP_PREFIX = "spark.hadoop." @@ -350,6 +363,8 @@ object GlutenConfig { // Pass through to native conf val GLUTEN_SAVE_DIR = "spark.gluten.saveDir" + val GLUTEN_DEBUG_MODE = "spark.gluten.sql.debug" + // Added back to Spark Conf during executor initialization val GLUTEN_OFFHEAP_SIZE_IN_BYTES_KEY = "spark.gluten.memory.offHeap.size.in.bytes" val GLUTEN_CONSERVATIVE_OFFHEAP_SIZE_IN_BYTES_KEY = @@ -414,12 +429,18 @@ object GlutenConfig { conf: scala.collection.Map[String, String]): util.Map[String, String] = { val nativeConfMap = new util.HashMap[String, String]() val keys = ImmutableList.of( + GLUTEN_DEBUG_MODE, GLUTEN_SAVE_DIR, GLUTEN_TASK_OFFHEAP_SIZE_IN_BYTES_KEY, GLUTEN_MAX_BATCH_SIZE_KEY, GLUTEN_SHUFFLE_WRITER_BUFFER_SIZE, SQLConf.SESSION_LOCAL_TIMEZONE.key, - GLUTEN_DEFAULT_SESSION_TIMEZONE_KEY + GLUTEN_DEFAULT_SESSION_TIMEZONE_KEY, + SQLConf.LEGACY_SIZE_OF_NULL.key, + "spark.io.compression.codec", + COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS.key, + COLUMNAR_VELOX_BLOOM_FILTER_NUM_BITS.key, + COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS.key ) keys.forEach( k => { @@ -481,11 +502,16 @@ object GlutenConfig { ("spark.hadoop.input.connect.timeout", "180000"), ("spark.hadoop.input.read.timeout", "180000"), ("spark.hadoop.input.write.timeout", "180000"), - ("spark.hadoop.dfs.client.log.severity", "INFO") + ("spark.hadoop.dfs.client.log.severity", "INFO"), + ("spark.sql.orc.compression.codec", "snappy"), + ( + COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED.key, + COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED.defaultValueString) ) keyWithDefault.forEach(e => nativeConfMap.put(e._1, conf.getOrElse(e._1, e._2))) val keys = ImmutableList.of( + GLUTEN_DEBUG_MODE, // datasource config SPARK_SQL_PARQUET_COMPRESSION_CODEC, // datasource config end @@ -603,6 +629,21 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val COLUMNAR_VELOX_WINDOW_TYPE = + buildConf("spark.gluten.sql.columnar.backend.velox.window.type") + .internal() + .doc( + "Velox backend supports both SortWindow and" + + " StreamingWindow operators." + + " The StreamingWindow operator skips the sorting step" + + " in the input but does not support spill." + + " On the other hand, the SortWindow operator is " + + "responsible for sorting the input data within the" + + " Window operator and also supports spill.") + .stringConf + .checkValues(Set("streaming", "sort")) + .createWithDefault("streaming") + val COLUMNAR_FPRCE_SHUFFLED_HASH_JOIN_ENABLED = buildConf("spark.gluten.sql.columnar.forceShuffledHashJoin") .internal() @@ -699,15 +740,7 @@ object GlutenConfig { .internal() .doc("Enable or disable columnar table cache.") .booleanConf - .createWithDefault(true) - - val COLUMNAR_ITERATOR_ENABLED = - buildConf("spark.gluten.sql.columnar.iterator") - .internal() - .doc( - "This config is used for specifying whether to use a columnar iterator in WS transformer.") - .booleanConf - .createWithDefault(true) + .createWithDefault(false) val COLUMNAR_PHYSICAL_JOIN_OPTIMIZATION_THROTTLE = buildConf("spark.gluten.sql.columnar.physicalJoinOptimizationLevel") @@ -799,11 +832,13 @@ object GlutenConfig { .checkValues(Set("buffer", "rowvector")) .createWithDefault("buffer") - val COLUMNAR_SHUFFLE_BUFFER_COMPRESS_THRESHOLD = - buildConf("spark.gluten.sql.columnar.shuffle.bufferCompressThreshold") + val COLUMNAR_SHUFFLE_COMPRESSION_THRESHOLD = + buildConf("spark.gluten.sql.columnar.shuffle.compression.threshold") .internal() + .doc("If number of rows in a batch falls below this threshold," + + " will copy all buffers into one buffer to compress.") .intConf - .createWithDefault(1024) + .createWithDefault(100) val COLUMNAR_MAX_BATCH_SIZE = buildConf(GLUTEN_MAX_BATCH_SIZE_KEY) @@ -1019,6 +1054,20 @@ object GlutenConfig { .intConf .createWithDefault(2) + val COLUMNAR_VELOX_GLOG_VERBOSE_LEVEL = + buildStaticConf("spark.gluten.sql.columnar.backend.velox.glogVerboseLevel") + .internal() + .doc("Set glog verbose level in Velox backend, same as FLAGS_v.") + .intConf + .createWithDefault(0) + + val COLUMNAR_VELOX_GLOG_SEVERITY_LEVEL = + buildStaticConf("spark.gluten.sql.columnar.backend.velox.glogSeverityLevel") + .internal() + .doc("Set glog severity level in Velox backend, same as FLAGS_minloglevel.") + .intConf + .createWithDefault(0) + val COLUMNAR_VELOX_SPILL_STRATEGY = buildConf("spark.gluten.sql.columnar.backend.velox.spillStrategy") .internal() @@ -1118,7 +1167,7 @@ object GlutenConfig { .createWithDefault("DEBUG") val DEBUG_LEVEL_ENABLED = - buildConf("spark.gluten.sql.debug") + buildConf(GLUTEN_DEBUG_MODE) .internal() .booleanConf .createWithDefault(false) @@ -1127,7 +1176,7 @@ object GlutenConfig { buildConf("spark.gluten.sql.benchmark_task.stageId") .internal() .intConf - .createWithDefault(1) + .createWithDefault(-1) val BENCHMARK_TASK_PARTITIONID = buildConf("spark.gluten.sql.benchmark_task.partitionId") @@ -1185,6 +1234,13 @@ object GlutenConfig { .booleanConf .createWithDefault(true) + val COLUMNAR_VELOX_MEMORY_USE_HUGE_PAGES = + buildConf("spark.gluten.sql.columnar.backend.velox.memoryUseHugePages") + .internal() + .doc("Use explicit huge pages for Velox memory allocation.") + .booleanConf + .createWithDefault(false) + val COLUMNAR_VELOX_ENABLE_SYSTEM_EXCEPTION_STACKTRACE = buildConf("spark.gluten.sql.columnar.backend.velox.enableSystemExceptionStacktrace") .internal() @@ -1248,4 +1304,60 @@ object GlutenConfig { + "partial aggregation may be early abandoned.") .intConf .createOptional + + val ENABLE_DATE_TIMESTAMP_COMPARISON = + buildConf("spark.gluten.sql.rewrite.dateTimestampComparison") + .internal() + .doc("Rewrite the comparision between date and timestamp to timestamp comparison." + + "For example `fron_unixtime(ts) > date` will be rewritten to `ts > to_unixtime(date)`") + .booleanConf + .createWithDefault(true) + + val COLUMNAR_VELOX_BLOOM_FILTER_EXPECTED_NUM_ITEMS = + buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.expectedNumItems") + .internal() + .doc("The default number of expected items for the velox bloomfilter: " + + "'spark.bloom_filter.expected_num_items'") + .longConf + .createWithDefault(1000000L) + + val COLUMNAR_VELOX_BLOOM_FILTER_NUM_BITS = + buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.numBits") + .internal() + .doc("The default number of bits to use for the velox bloom filter: " + + "'spark.bloom_filter.num_bits'") + .longConf + .createWithDefault(8388608L) + + val COLUMNAR_VELOX_BLOOM_FILTER_MAX_NUM_BITS = + buildConf("spark.gluten.sql.columnar.backend.velox.bloomFilter.maxNumBits") + .internal() + .doc("The max number of bits to use for the velox bloom filter: " + + "'spark.bloom_filter.max_num_bits'") + .longConf + .createWithDefault(4194304L) + + val COLUMNAR_VELOX_FILE_HANDLE_CACHE_ENABLED = + buildStaticConf("spark.gluten.sql.columnar.backend.velox.fileHandleCacheEnabled") + .internal() + .doc("Disables caching if false. File handle cache should be disabled " + + "if files are mutable, i.e. file content may change while file path stays the same.") + .booleanConf + .createWithDefault(false) + + val CACHE_WHOLE_STAGE_TRANSFORMER_CONTEXT = + buildConf("spark.gluten.sql.cacheWholeStageTransformerContext") + .internal() + .doc("When true, `WholeStageTransformer` will cache the `WholeStageTransformerContext` " + + "when executing. It is used to get substrait plan node and native plan string.") + .booleanConf + .createWithDefault(false) + + val INJECT_NATIVE_PLAN_STRING_TO_EXPLAIN = + buildConf("spark.gluten.sql.injectNativePlanStringToExplain") + .internal() + .doc("When true, Gluten will inject native plan tree to explain string inside " + + "`WholeStageTransformerContext`.") + .booleanConf + .createWithDefault(false) } diff --git a/shims/common/src/main/scala/io/glutenproject/expression/ExpressionNames.scala b/shims/common/src/main/scala/io/glutenproject/expression/ExpressionNames.scala index 385ce3269de7..80e2cf169c85 100644 --- a/shims/common/src/main/scala/io/glutenproject/expression/ExpressionNames.scala +++ b/shims/common/src/main/scala/io/glutenproject/expression/ExpressionNames.scala @@ -24,6 +24,8 @@ object ExpressionNames { final val COUNT = "count" final val MIN = "min" final val MAX = "max" + final val MAX_BY = "max_by" + final val MIN_BY = "min_by" final val STDDEV_SAMP = "stddev_samp" final val STDDEV_POP = "stddev_pop" final val COLLECT_LIST = "collect_list" @@ -96,6 +98,7 @@ object ExpressionNames { final val SUBSTRING = "substring" final val SUBSTRING_INDEX = "substring_index" final val CONCAT_WS = "concat_ws" + final val LEFT = "left" final val REPEAT = "repeat" final val TRANSLATE = "translate" final val SPACE = "space" diff --git a/gluten-core/src/main/scala/io/glutenproject/metrics/GlutenTimeMetric.scala b/shims/common/src/main/scala/io/glutenproject/metrics/GlutenTimeMetric.scala similarity index 100% rename from gluten-core/src/main/scala/io/glutenproject/metrics/GlutenTimeMetric.scala rename to shims/common/src/main/scala/io/glutenproject/metrics/GlutenTimeMetric.scala diff --git a/shims/common/src/main/scala/io/glutenproject/sql/shims/SparkShims.scala b/shims/common/src/main/scala/io/glutenproject/sql/shims/SparkShims.scala index 5a00c6e52bba..de99e7efb44c 100644 --- a/shims/common/src/main/scala/io/glutenproject/sql/shims/SparkShims.scala +++ b/shims/common/src/main/scala/io/glutenproject/sql/shims/SparkShims.scala @@ -21,12 +21,15 @@ import io.glutenproject.expression.Sig import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Expression, PlanExpression} import org.apache.spark.sql.catalyst.plans.physical.Distribution +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.text.TextScan +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -67,4 +70,15 @@ trait SparkShims { def filesGroupedToBuckets( selectedPartitions: Array[PartitionDirectory]): Map[Int, Array[PartitionedFile]] + + // Spark3.4 new add table parameter in BatchScanExec. + def getBatchScanExecTable(batchScan: BatchScanExec): Table + + // The PartitionedFile API changed in spark 3.4 + def generatePartitionedFile( + partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long, + @transient locations: Array[String] = Array.empty): PartitionedFile } diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/Arm.scala b/shims/common/src/main/scala/io/glutenproject/utils/Arm.scala similarity index 100% rename from gluten-core/src/main/scala/io/glutenproject/utils/Arm.scala rename to shims/common/src/main/scala/io/glutenproject/utils/Arm.scala diff --git a/shims/pom.xml b/shims/pom.xml index 683790549663..78fa612d8b71 100644 --- a/shims/pom.xml +++ b/shims/pom.xml @@ -28,6 +28,10 @@ Gluten Shims pom + + common + + org.scalatest @@ -46,7 +50,6 @@ org.apache.hadoop hadoop-client - ${hadoop.version} @@ -64,25 +67,26 @@ - + spark-3.2 - - true - - common spark32 spark-3.3 - common spark33 + + spark-3.4 + + spark34 + + diff --git a/shims/spark32/pom.xml b/shims/spark32/pom.xml index b46255d76067..44eb72dce289 100644 --- a/shims/spark32/pom.xml +++ b/shims/spark32/pom.xml @@ -38,21 +38,18 @@ org.apache.spark spark-sql_${scala.binary.version} - ${spark32.version} provided true org.apache.spark spark-catalyst_2.12 - ${spark32.version} provided true org.apache.spark spark-core_2.12 - ${spark32.version} provided true @@ -72,25 +69,21 @@ org.apache.spark spark-core_${scala.binary.version} - ${spark32.version} test-jar org.apache.spark spark-sql_${scala.binary.version} - ${spark32.version} test-jar org.apache.spark spark-catalyst_${scala.binary.version} - ${spark32.version} test-jar org.apache.spark spark-hive_${scala.binary.version} - ${spark.version} provided diff --git a/shims/spark32/src/main/scala/io/glutenproject/sql/shims/spark32/Spark32Shims.scala b/shims/spark32/src/main/scala/io/glutenproject/sql/shims/spark32/Spark32Shims.scala index db3cdf56bbc6..580cc93bdf75 100644 --- a/shims/spark32/src/main/scala/io/glutenproject/sql/shims/spark32/Spark32Shims.scala +++ b/shims/spark32/src/main/scala/io/glutenproject/sql/shims/spark32/Spark32Shims.scala @@ -16,7 +16,7 @@ */ package io.glutenproject.sql.shims.spark32 -import io.glutenproject.expression.Sig +import io.glutenproject.expression.{ExpressionNames, Sig} import io.glutenproject.sql.shims.{ShimDescriptor, SparkShims} import org.apache.spark.sql.SparkSession @@ -24,9 +24,12 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.physical.{Distribution, HashClusteredDistribution} +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil} import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, FileScanRDD, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.FileFormatWriter.Empty2Null +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.text.TextScan import org.apache.spark.sql.execution.datasources.v2.utils.CatalogUtil import org.apache.spark.sql.types.StructType @@ -41,7 +44,7 @@ class Spark32Shims extends SparkShims { HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil } - override def expressionMappings: Seq[Sig] = Seq.empty + override def expressionMappings: Seq[Sig] = Seq(Sig[Empty2Null](ExpressionNames.EMPTY2NULL)) override def convertPartitionTransforms( partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { @@ -88,4 +91,14 @@ class Spark32Shims extends SparkShims { .getOrElse(throw new IllegalStateException(s"Invalid bucket file ${f.filePath}")) } } + + override def getBatchScanExecTable(batchScan: BatchScanExec): Table = null + + override def generatePartitionedFile( + partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long, + @transient locations: Array[String] = Array.empty): PartitionedFile = + PartitionedFile(partitionValues, filePath, start, length, locations) } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.scala new file mode 100644 index 000000000000..bc7cacf7995e --- /dev/null +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.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.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, IntegerLiteral} + +/** + * A logical offset, which may removing a specified number of rows from the beginning of the output + * of child logical plan. + */ +case class Offset(offsetExpr: Expression, child: LogicalPlan) extends OrderPreservingUnaryNode { + override def output: Seq[Attribute] = child.output + override def maxRows: Option[Long] = { + import scala.math.max + offsetExpr match { + case IntegerLiteral(offset) => child.maxRows.map(x => max(x - offset, 0)) + case _ => None + } + } + override protected def withNewChildInternal(newChild: LogicalPlan): Offset = + copy(child = newChild) +} diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala index c721a4beda01..9e32b35b8f3b 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala @@ -16,13 +16,19 @@ */ package org.apache.spark.sql.execution +import io.glutenproject.metrics.GlutenTimeMetric + import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BoundReference, DynamicPruningExpression, Expression, PlanExpression, Predicate} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.StructType import org.apache.spark.util.collection.BitSet +import java.util.concurrent.TimeUnit.NANOSECONDS + +import scala.collection.mutable + class FileSourceScanExecShim( @transient relation: HadoopFsRelation, output: Seq[Attribute], @@ -60,4 +66,95 @@ class FileSourceScanExecShim( def hasMetadataColumns: Boolean = false def hasFieldIds: Boolean = false + + // The codes below are copied from FileSourceScanExec in Spark, + // all of them are private. + protected lazy val driverMetrics: mutable.HashMap[String, Long] = mutable.HashMap.empty + + /** + * Send the driver-side metrics. Before calling this function, selectedPartitions has been + * initialized. See SPARK-26327 for more details. + */ + protected def sendDriverMetrics(): Unit = { + driverMetrics.foreach(e => metrics(e._1).add(e._2)) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkContext, + executionId, + metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) + } + + protected def setFilesNumAndSizeMetric( + partitions: Seq[PartitionDirectory], + static: Boolean): Unit = { + val filesNum = partitions.map(_.files.size.toLong).sum + val filesSize = partitions.map(_.files.map(_.getLen).sum).sum + if (!static || !partitionFilters.exists(isDynamicPruningFilter)) { + driverMetrics("numFiles") = filesNum + driverMetrics("filesSize") = filesSize + } else { + driverMetrics("staticFilesNum") = filesNum + driverMetrics("staticFilesSize") = filesSize + } + if (relation.partitionSchema.nonEmpty) { + driverMetrics("numPartitions") = partitions.length + } + } + + @transient override lazy val selectedPartitions: Array[PartitionDirectory] = { + val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) + GlutenTimeMetric.withNanoTime { + val ret = + relation.location.listFiles(partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) + setFilesNumAndSizeMetric(ret, static = true) + ret + }(t => driverMetrics("metadataTime") = NANOSECONDS.toMillis(t + optimizerMetadataTimeNs)) + }.toArray + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.find(_.isInstanceOf[PlanExpression[_]]).isDefined + + // We can only determine the actual partitions at runtime when a dynamic partition filter is + // present. This is because such a filter relies on information that is only available at run + // time (for instance the keys used in the other side of a join). + @transient lazy val dynamicallySelectedPartitions: Array[PartitionDirectory] = { + val dynamicPartitionFilters = + partitionFilters.filter(isDynamicPruningFilter) + val selected = if (dynamicPartitionFilters.nonEmpty) { + // When it includes some DynamicPruningExpression, + // it needs to execute InSubqueryExec first, + // because doTransform path can't execute 'doExecuteColumnar' which will + // execute prepare subquery first. + dynamicPartitionFilters.foreach { + case DynamicPruningExpression(inSubquery: InSubqueryExec) => + if (inSubquery.values().isEmpty) inSubquery.updateResult() + case e: Expression => + e.foreach { + case s: ScalarSubquery => s.updateResult() + case _ => + } + case _ => + } + GlutenTimeMetric.withMillisTime { + // call the file index for the files matching all filters except dynamic partition filters + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil + ) + val ret = selectedPartitions.filter(p => boundPredicate.eval(p.values)) + setFilesNumAndSizeMetric(ret, static = false) + ret + }(t => driverMetrics("pruningTime") = t) + } else { + selectedPartitions + } + sendDriverMetrics() + selected + } } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index f468ad36b824..d5376f8d1b72 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -407,19 +407,21 @@ class DynamicPartitionDataSingleWriter( override def write(record: InternalRow): Unit = { record match { case fakeRow: FakeRow => - val blockStripes = GlutenRowSplitter.getInstance - .splitBlockByPartitionAndBucket(fakeRow, partitionColIndice, isBucketed) - - val iter = blockStripes.iterator(); - while (iter.hasNext) { - val blockStripe = iter.next() - val headingRow = blockStripe.getHeadingRow - beforeWrite(headingRow) - val columnBatch = blockStripe.getColumnarBatch - writeStripe(new FakeRow(columnBatch)) - columnBatch.close() + if (fakeRow.batch.numRows() > 0) { + val blockStripes = GlutenRowSplitter.getInstance + .splitBlockByPartitionAndBucket(fakeRow, partitionColIndice, isBucketed) + + val iter = blockStripes.iterator() + while (iter.hasNext) { + val blockStripe = iter.next() + val headingRow = blockStripe.getHeadingRow + beforeWrite(headingRow) + val columnBatch = blockStripe.getColumnarBatch + writeStripe(new FakeRow(columnBatch)) + columnBatch.close() + } + blockStripes.release() } - blockStripes.release() case _ => beforeWrite(record) writeRecord(record) diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index ccf20540093c..fb57ebd982f5 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -108,6 +108,32 @@ object FileFormatWriter extends Logging { * @return * The set of all partition paths that were updated during this write job. */ + + // scalastyle:off argcount + def write( + sparkSession: SparkSession, + plan: SparkPlan, + fileFormat: FileFormat, + committer: FileCommitProtocol, + outputSpec: OutputSpec, + hadoopConf: Configuration, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + statsTrackers: Seq[WriteJobStatsTracker], + options: Map[String, String]): Set[String] = write( + sparkSession = sparkSession, + plan = plan, + fileFormat = fileFormat, + committer = committer, + outputSpec = outputSpec, + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = bucketSpec, + statsTrackers = statsTrackers, + options = options, + numStaticPartitionCols = 0 + ) + def write( sparkSession: SparkSession, plan: SparkPlan, @@ -118,7 +144,8 @@ object FileFormatWriter extends Logging { partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], statsTrackers: Seq[WriteJobStatsTracker], - options: Map[String, String]): Set[String] = { + options: Map[String, String], + numStaticPartitionCols: Int = 0): Set[String] = { val nativeEnabled = "true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) @@ -195,7 +222,8 @@ object FileFormatWriter extends Logging { ) // We should first sort by partition columns, then bucket id, and finally sorting columns. - val requiredOrdering = partitionColumns ++ bucketIdExpression ++ sortColumns + val requiredOrdering = partitionColumns.drop(numStaticPartitionCols) ++ + bucketIdExpression ++ sortColumns // the sort order doesn't matter val actualOrdering = empty2NullPlan.outputOrdering.map(_.child) val orderingMatched = if (requiredOrdering.length > actualOrdering.length) { @@ -328,6 +356,7 @@ object FileFormatWriter extends Logging { throw QueryExecutionErrors.jobAbortedError(cause) } } + // scalastyle:on argcount /** Writes data out in a single Spark task. */ private def executeTask( diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala new file mode 100644 index 000000000000..9221ecbd1294 --- /dev/null +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -0,0 +1,288 @@ +/* + * 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.datasources + +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode +import org.apache.spark.sql.util.SchemaUtils + +import org.apache.hadoop.fs.{FileSystem, Path} + +/** + * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. + * Writing to dynamic partitions is also supported. + * + * @param staticPartitions + * partial partitioning spec for write. This defines the scope of partition overwrites: when the + * spec is empty, all partitions are overwritten. When it covers a prefix of the partition keys, + * only partitions matching the prefix are overwritten. + * @param ifPartitionNotExists + * If true, only write if the partition does not exist. Only valid for static partitions. + */ +case class InsertIntoHadoopFsRelationCommand( + outputPath: Path, + staticPartitions: TablePartitionSpec, + ifPartitionNotExists: Boolean, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + fileFormat: FileFormat, + options: Map[String, String], + query: LogicalPlan, + mode: SaveMode, + catalogTable: Option[CatalogTable], + fileIndex: Option[FileIndex], + outputColumnNames: Seq[String]) + extends DataWritingCommand { + + private lazy val parameters = CaseInsensitiveMap(options) + + private[sql] lazy val dynamicPartitionOverwrite: Boolean = { + val partitionOverwriteMode = parameters + .get("partitionOverwriteMode") + // scalastyle:off caselocale + .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase)) + // scalastyle:on caselocale + .getOrElse(conf.partitionOverwriteMode) + val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC + // This config only makes sense when we are overwriting a partitioned dataset with dynamic + // partition columns. + enableDynamicOverwrite && mode == SaveMode.Overwrite && + staticPartitions.size < partitionColumns.length + } + + override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { + // Most formats don't do well with duplicate columns, so lets not allow that + SchemaUtils.checkColumnNameDuplication( + outputColumnNames, + s"when inserting into $outputPath", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) + val fs = outputPath.getFileSystem(hadoopConf) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + + val partitionsTrackedByCatalog = sparkSession.sessionState.conf.manageFilesourcePartitions && + catalogTable.isDefined && + catalogTable.get.partitionColumnNames.nonEmpty && + catalogTable.get.tracksPartitionsInCatalog + + var initialMatchingPartitions: Seq[TablePartitionSpec] = Nil + var customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty + var matchingPartitions: Seq[CatalogTablePartition] = Seq.empty + + // When partitions are tracked by the catalog, compute all custom partition locations that + // may be relevant to the insertion job. + if (partitionsTrackedByCatalog) { + matchingPartitions = sparkSession.sessionState.catalog + .listPartitions(catalogTable.get.identifier, Some(staticPartitions)) + initialMatchingPartitions = matchingPartitions.map(_.spec) + customPartitionLocations = + getCustomPartitionLocations(fs, catalogTable.get, qualifiedOutputPath, matchingPartitions) + } + + val jobId = java.util.UUID.randomUUID().toString + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = jobId, + outputPath = outputPath.toString, + dynamicPartitionOverwrite = dynamicPartitionOverwrite) + + val doInsertion = if (mode == SaveMode.Append) { + true + } else { + val pathExists = fs.exists(qualifiedOutputPath) + (mode, pathExists) match { + case (SaveMode.ErrorIfExists, true) => + throw QueryCompilationErrors.outputPathAlreadyExistsError(qualifiedOutputPath) + case (SaveMode.Overwrite, true) => + if (ifPartitionNotExists && matchingPartitions.nonEmpty) { + false + } else if (dynamicPartitionOverwrite) { + // For dynamic partition overwrite, do not delete partition directories ahead. + true + } else { + deleteMatchingPartitions(fs, qualifiedOutputPath, customPartitionLocations, committer) + true + } + case (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => + true + case (SaveMode.Ignore, exists) => + !exists + case (s, exists) => + throw QueryExecutionErrors.unsupportedSaveModeError(s.toString, exists) + } + } + + if (doInsertion) { + + def refreshUpdatedPartitions(updatedPartitionPaths: Set[String]): Unit = { + val updatedPartitions = updatedPartitionPaths.map(PartitioningUtils.parsePathFragment) + if (partitionsTrackedByCatalog) { + val newPartitions = updatedPartitions -- initialMatchingPartitions + if (newPartitions.nonEmpty) { + AlterTableAddPartitionCommand( + catalogTable.get.identifier, + newPartitions.toSeq.map(p => (p, None)), + ifNotExists = true).run(sparkSession) + } + // For dynamic partition overwrite, we never remove partitions but only update existing + // ones. + if (mode == SaveMode.Overwrite && !dynamicPartitionOverwrite) { + val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions + if (deletedPartitions.nonEmpty) { + AlterTableDropPartitionCommand( + catalogTable.get.identifier, + deletedPartitions.toSeq, + ifExists = true, + purge = false, + retainData = true /* already deleted */ ).run(sparkSession) + } + } + } + } + + // For dynamic partition overwrite, FileOutputCommitter's output path is staging path, files + // will be renamed from staging path to final output path during commit job + val committerOutputPath = if (dynamicPartitionOverwrite) { + FileCommitProtocol + .getStagingDir(outputPath.toString, jobId) + .makeQualified(fs.getUri, fs.getWorkingDirectory) + } else { + qualifiedOutputPath + } + + val updatedPartitionPaths = + FileFormatWriter.write( + sparkSession = sparkSession, + plan = child, + fileFormat = fileFormat, + committer = committer, + outputSpec = FileFormatWriter.OutputSpec( + committerOutputPath.toString, + customPartitionLocations, + outputColumns), + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = bucketSpec, + statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), + options = options, + numStaticPartitionCols = staticPartitions.size + ) + + // update metastore partition metadata + if ( + updatedPartitionPaths.isEmpty && staticPartitions.nonEmpty + && partitionColumns.length == staticPartitions.size + ) { + // Avoid empty static partition can't loaded to datasource table. + val staticPathFragment = + PartitioningUtils.getPathFragment(staticPartitions, partitionColumns) + refreshUpdatedPartitions(Set(staticPathFragment)) + } else { + refreshUpdatedPartitions(updatedPartitionPaths) + } + + // refresh cached files in FileIndex + fileIndex.foreach(_.refresh()) + // refresh data cache if table is cached + sparkSession.sharedState.cacheManager.recacheByPath(sparkSession, outputPath, fs) + + if (catalogTable.nonEmpty) { + CommandUtils.updateTableStats(sparkSession, catalogTable.get) + } + + } else { + logInfo("Skipping insertion into a relation that already exists.") + } + + Seq.empty[Row] + } + + /** + * Deletes all partition files that match the specified static prefix. Partitions with custom + * locations are also cleared based on the custom locations map given to this class. + */ + private def deleteMatchingPartitions( + fs: FileSystem, + qualifiedOutputPath: Path, + customPartitionLocations: Map[TablePartitionSpec, String], + committer: FileCommitProtocol): Unit = { + val staticPartitionPrefix = if (staticPartitions.nonEmpty) { + "/" + partitionColumns + .flatMap(p => staticPartitions.get(p.name).map(getPartitionPathString(p.name, _))) + .mkString("/") + } else { + "" + } + // first clear the path determined by the static partition keys (e.g. /table/foo=1) + val staticPrefixPath = qualifiedOutputPath.suffix(staticPartitionPrefix) + if (fs.exists(staticPrefixPath) && !committer.deleteWithJob(fs, staticPrefixPath, true)) { + throw QueryExecutionErrors.cannotClearOutputDirectoryError(staticPrefixPath) + } + // now clear all custom partition locations (e.g. /custom/dir/where/foo=2/bar=4) + for ((spec, customLoc) <- customPartitionLocations) { + assert( + (staticPartitions.toSet -- spec).isEmpty, + "Custom partition location did not match static partitioning keys") + val path = new Path(customLoc) + if (fs.exists(path) && !committer.deleteWithJob(fs, path, true)) { + throw QueryExecutionErrors.cannotClearPartitionDirectoryError(path) + } + } + } + + /** + * Given a set of input partitions, returns those that have locations that differ from the Hive + * default (e.g. /k1=v1/k2=v2). These partitions were manually assigned locations by the user. + * + * @return + * a mapping from partition specs to their custom locations + */ + private def getCustomPartitionLocations( + fs: FileSystem, + table: CatalogTable, + qualifiedOutputPath: Path, + partitions: Seq[CatalogTablePartition]): Map[TablePartitionSpec, String] = { + partitions.flatMap { + p => + val defaultLocation = qualifiedOutputPath + .suffix("/" + PartitioningUtils.getPathFragment(p.spec, table.partitionSchema)) + .toString + val catalogLocation = + new Path(p.location).makeQualified(fs.getUri, fs.getWorkingDirectory).toString + if (catalogLocation != defaultLocation) { + Some(p.spec -> catalogLocation) + } else { + None + } + }.toMap + } + + override protected def withNewChildInternal( + newChild: LogicalPlan): InsertIntoHadoopFsRelationCommand = copy(query = newChild) +} +// scalastyle:on line.size.limit diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala index 112b5832d95f..b867c71cefda 100644 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala +++ b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala @@ -16,11 +16,10 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import io.glutenproject.GlutenConfig - import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{InputPartition, Scan, SupportsRuntimeFiltering} import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -30,14 +29,13 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class BatchScanExecShim( output: Seq[AttributeReference], @transient scan: Scan, - runtimeFilters: Seq[Expression]) + runtimeFilters: Seq[Expression], + @transient table: Table) extends BatchScanExec(output, scan, runtimeFilters) { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics: Map[String, SQLMetric] = Map() - override def supportsColumnar(): Boolean = GlutenConfig.getConf.enableColumnarIterator - override def doExecuteColumnar(): RDD[ColumnarBatch] = { throw new UnsupportedOperationException("Need to implement this method") } @@ -86,4 +84,8 @@ class BatchScanExecShim( } @transient lazy val pushedAggregate: Option[Aggregation] = None + + final override protected def otherCopyArgs: Seq[AnyRef] = { + output :: scan :: runtimeFilters :: Nil + } } diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala deleted file mode 100644 index 621486d743e5..000000000000 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala +++ /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. - */ -package org.apache.spark.sql.execution.datasources.v2.velox - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.connector.read.PartitionReaderFactory -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class DwrfScan( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - readDataSchema: StructType, - readPartitionSchema: StructType, - pushedFilters: Array[Filter], - options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty, - dataFilters: Seq[Expression] = Seq.empty) - extends FileScan { - override def createReaderFactory(): PartitionReaderFactory = { - null - } - - override def withFilters( - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) -} diff --git a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala deleted file mode 100644 index dda9aeca7a47..000000000000 --- a/shims/spark32/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala +++ /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. - */ -package org.apache.spark.sql.execution.datasources.v2.velox - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class DwrfScanBuilder( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - schema: StructType, - dataSchema: StructType, - options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) - with SupportsPushDownFilters { - - private lazy val pushedArrowFilters: Array[Filter] = { - filters // todo filter validation & pushdown - } - private var filters: Array[Filter] = Array.empty - - override def pushFilters(filters: Array[Filter]): Array[Filter] = { - this.filters = filters - this.filters - } - - override def build(): Scan = { - DwrfScan( - sparkSession, - fileIndex, - readDataSchema(), - readPartitionSchema(), - pushedFilters, - options) - } - - override def pushedFilters: Array[Filter] = pushedArrowFilters -} diff --git a/shims/common/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/shims/spark32/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala similarity index 100% rename from shims/common/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala rename to shims/spark32/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala diff --git a/shims/spark33/pom.xml b/shims/spark33/pom.xml index 075610347ffc..c9bf97c2fa85 100644 --- a/shims/spark33/pom.xml +++ b/shims/spark33/pom.xml @@ -38,21 +38,18 @@ org.apache.spark spark-sql_${scala.binary.version} - ${spark33.version} provided true org.apache.spark spark-catalyst_2.12 - ${spark33.version} provided true org.apache.spark spark-core_2.12 - ${spark33.version} provided true @@ -72,25 +69,21 @@ org.apache.spark spark-core_${scala.binary.version} - ${spark33.version} test-jar org.apache.spark spark-sql_${scala.binary.version} - ${spark33.version} test-jar org.apache.spark spark-catalyst_${scala.binary.version} - ${spark33.version} test-jar org.apache.spark spark-hive_${scala.binary.version} - ${spark.version} provided diff --git a/shims/spark33/src/main/scala/io/glutenproject/sql/shims/spark33/Spark33Shims.scala b/shims/spark33/src/main/scala/io/glutenproject/sql/shims/spark33/Spark33Shims.scala index 06325480f244..50e536610e7b 100644 --- a/shims/spark33/src/main/scala/io/glutenproject/sql/shims/spark33/Spark33Shims.scala +++ b/shims/spark33/src/main/scala/io/glutenproject/sql/shims/spark33/Spark33Shims.scala @@ -27,9 +27,12 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution} +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.execution.{FileSourceScanExec, PartitionedFileUtil, SparkPlan} import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, FileScanRDD, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.FileFormatWriter.Empty2Null +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.text.TextScan import org.apache.spark.sql.execution.datasources.v2.utils.CatalogUtil import org.apache.spark.sql.types.StructType @@ -55,7 +58,8 @@ class Spark33Shims extends SparkShims { list ++ Seq( Sig[SplitPart](ExpressionNames.SPLIT_PART), Sig[Sec](ExpressionNames.SEC), - Sig[Csc](ExpressionNames.CSC)) + Sig[Csc](ExpressionNames.CSC), + Sig[Empty2Null](ExpressionNames.EMPTY2NULL)) } override def convertPartitionTransforms( @@ -113,6 +117,16 @@ class Spark33Shims extends SparkShims { } } + override def getBatchScanExecTable(batchScan: BatchScanExec): Table = null + + override def generatePartitionedFile( + partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long, + @transient locations: Array[String] = Array.empty): PartitionedFile = + PartitionedFile(partitionValues, filePath, start, length, locations) + private def invalidBucketFile(path: String): Throwable = { new SparkException( errorClass = "INVALID_BUCKET_FILE", diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.scala new file mode 100644 index 000000000000..bc7cacf7995e --- /dev/null +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Offset.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.spark.sql.catalyst.plans.logical + +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, IntegerLiteral} + +/** + * A logical offset, which may removing a specified number of rows from the beginning of the output + * of child logical plan. + */ +case class Offset(offsetExpr: Expression, child: LogicalPlan) extends OrderPreservingUnaryNode { + override def output: Seq[Attribute] = child.output + override def maxRows: Option[Long] = { + import scala.math.max + offsetExpr match { + case IntegerLiteral(offset) => child.maxRows.map(x => max(x - offset, 0)) + case _ => None + } + } + override protected def withNewChildInternal(newChild: LogicalPlan): Offset = + copy(child = newChild) +} diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala index 88bd259c6e10..cfbf91bc2188 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala @@ -16,14 +16,20 @@ */ package org.apache.spark.sql.execution +import io.glutenproject.metrics.GlutenTimeMetric + import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BoundReference, DynamicPruningExpression, Expression, PlanExpression, Predicate} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.types.StructType import org.apache.spark.util.collection.BitSet +import java.util.concurrent.TimeUnit.NANOSECONDS + +import scala.collection.mutable + class FileSourceScanExecShim( @transient relation: HadoopFsRelation, output: Seq[Attribute], @@ -61,4 +67,95 @@ class FileSourceScanExecShim( def hasMetadataColumns: Boolean = metadataColumns.nonEmpty def hasFieldIds: Boolean = ParquetUtils.hasFieldIds(requiredSchema) + + // The codes below are copied from FileSourceScanExec in Spark, + // all of them are private. + protected lazy val driverMetrics: mutable.HashMap[String, Long] = mutable.HashMap.empty + + /** + * Send the driver-side metrics. Before calling this function, selectedPartitions has been + * initialized. See SPARK-26327 for more details. + */ + protected def sendDriverMetrics(): Unit = { + driverMetrics.foreach(e => metrics(e._1).add(e._2)) + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates( + sparkContext, + executionId, + metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) + } + + protected def setFilesNumAndSizeMetric( + partitions: Seq[PartitionDirectory], + static: Boolean): Unit = { + val filesNum = partitions.map(_.files.size.toLong).sum + val filesSize = partitions.map(_.files.map(_.getLen).sum).sum + if (!static || !partitionFilters.exists(isDynamicPruningFilter)) { + driverMetrics("numFiles") = filesNum + driverMetrics("filesSize") = filesSize + } else { + driverMetrics("staticFilesNum") = filesNum + driverMetrics("staticFilesSize") = filesSize + } + if (relation.partitionSchema.nonEmpty) { + driverMetrics("numPartitions") = partitions.length + } + } + + @transient override lazy val selectedPartitions: Array[PartitionDirectory] = { + val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) + GlutenTimeMetric.withNanoTime { + val ret = + relation.location.listFiles(partitionFilters.filterNot(isDynamicPruningFilter), dataFilters) + setFilesNumAndSizeMetric(ret, static = true) + ret + }(t => driverMetrics("metadataTime") = NANOSECONDS.toMillis(t + optimizerMetadataTimeNs)) + }.toArray + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.find(_.isInstanceOf[PlanExpression[_]]).isDefined + + // We can only determine the actual partitions at runtime when a dynamic partition filter is + // present. This is because such a filter relies on information that is only available at run + // time (for instance the keys used in the other side of a join). + @transient lazy val dynamicallySelectedPartitions: Array[PartitionDirectory] = { + val dynamicPartitionFilters = + partitionFilters.filter(isDynamicPruningFilter) + val selected = if (dynamicPartitionFilters.nonEmpty) { + // When it includes some DynamicPruningExpression, + // it needs to execute InSubqueryExec first, + // because doTransform path can't execute 'doExecuteColumnar' which will + // execute prepare subquery first. + dynamicPartitionFilters.foreach { + case DynamicPruningExpression(inSubquery: InSubqueryExec) => + if (inSubquery.values().isEmpty) inSubquery.updateResult() + case e: Expression => + e.foreach { + case s: ScalarSubquery => s.updateResult() + case _ => + } + case _ => + } + GlutenTimeMetric.withMillisTime { + // call the file index for the files matching all filters except dynamic partition filters + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil + ) + val ret = selectedPartitions.filter(p => boundPredicate.eval(p.values)) + setFilesNumAndSizeMetric(ret, static = false) + ret + }(t => driverMetrics("pruningTime") = t) + } else { + selectedPartitions + } + sendDriverMetrics() + selected + } } diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 4ec65a0cfde4..c9beb1fde2d8 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -409,19 +409,21 @@ class DynamicPartitionDataSingleWriter( override def write(record: InternalRow): Unit = { record match { case fakeRow: FakeRow => - val blockStripes = GlutenRowSplitter.getInstance - .splitBlockByPartitionAndBucket(fakeRow, partitionColIndice, isBucketed) - - val iter = blockStripes.iterator(); - while (iter.hasNext) { - val blockStripe = iter.next() - val headingRow = blockStripe.getHeadingRow - beforeWrite(headingRow) - val columnBatch = blockStripe.getColumnarBatch - writeStripe(new FakeRow(columnBatch)) - columnBatch.close() + if (fakeRow.batch.numRows() > 0) { + val blockStripes = GlutenRowSplitter.getInstance + .splitBlockByPartitionAndBucket(fakeRow, partitionColIndice, isBucketed) + + val iter = blockStripes.iterator() + while (iter.hasNext) { + val blockStripe = iter.next() + val headingRow = blockStripe.getHeadingRow + beforeWrite(headingRow) + val columnBatch = blockStripe.getColumnarBatch + writeStripe(new FakeRow(columnBatch)) + columnBatch.close() + } + blockStripes.release() } - blockStripes.release() case _ => beforeWrite(record) writeRecord(record) diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 530f477ab742..0e6ea140b05e 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -100,6 +100,32 @@ object FileFormatWriter extends Logging { * @return * The set of all partition paths that were updated during this write job. */ + + // scalastyle:off argcount + def write( + sparkSession: SparkSession, + plan: SparkPlan, + fileFormat: FileFormat, + committer: FileCommitProtocol, + outputSpec: OutputSpec, + hadoopConf: Configuration, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + statsTrackers: Seq[WriteJobStatsTracker], + options: Map[String, String]): Set[String] = write( + sparkSession = sparkSession, + plan = plan, + fileFormat = fileFormat, + committer = committer, + outputSpec = outputSpec, + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = bucketSpec, + statsTrackers = statsTrackers, + options = options, + numStaticPartitionCols = 0 + ) + def write( sparkSession: SparkSession, plan: SparkPlan, @@ -110,7 +136,8 @@ object FileFormatWriter extends Logging { partitionColumns: Seq[Attribute], bucketSpec: Option[BucketSpec], statsTrackers: Seq[WriteJobStatsTracker], - options: Map[String, String]): Set[String] = { + options: Map[String, String], + numStaticPartitionCols: Int = 0): Set[String] = { val nativeEnabled = "true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable")) @@ -215,8 +242,8 @@ object FileFormatWriter extends Logging { ) // We should first sort by partition columns, then bucket id, and finally sorting columns. - val requiredOrdering = - partitionColumns ++ writerBucketSpec.map(_.bucketIdExpression) ++ sortColumns + val requiredOrdering = partitionColumns.drop(numStaticPartitionCols) ++ + writerBucketSpec.map(_.bucketIdExpression) ++ sortColumns // the sort order doesn't matter val actualOrdering = empty2NullPlan.outputOrdering.map(_.child) val orderingMatched = if (requiredOrdering.length > actualOrdering.length) { @@ -349,6 +376,7 @@ object FileFormatWriter extends Logging { throw QueryExecutionErrors.jobAbortedError(cause) } } + // scalastyle:on argcount /** Writes data out in a single Spark task. */ private def executeTask( diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala new file mode 100644 index 000000000000..b1e740284b56 --- /dev/null +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -0,0 +1,291 @@ +/* + * 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.datasources + +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTablePartition} +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode +import org.apache.spark.sql.util.SchemaUtils + +import org.apache.hadoop.fs.{FileSystem, Path} + +/** + * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. + * Writing to dynamic partitions is also supported. + * + * @param staticPartitions + * partial partitioning spec for write. This defines the scope of partition overwrites: when the + * spec is empty, all partitions are overwritten. When it covers a prefix of the partition keys, + * only partitions matching the prefix are overwritten. + * @param ifPartitionNotExists + * If true, only write if the partition does not exist. Only valid for static partitions. + */ + +// scalastyle:off line.size.limit +case class InsertIntoHadoopFsRelationCommand( + outputPath: Path, + staticPartitions: TablePartitionSpec, + ifPartitionNotExists: Boolean, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + fileFormat: FileFormat, + options: Map[String, String], + query: LogicalPlan, + mode: SaveMode, + catalogTable: Option[CatalogTable], + fileIndex: Option[FileIndex], + outputColumnNames: Seq[String]) + extends DataWritingCommand { + + private lazy val parameters = CaseInsensitiveMap(options) + + private[sql] lazy val dynamicPartitionOverwrite: Boolean = { + val partitionOverwriteMode = parameters + .get(DataSourceUtils.PARTITION_OVERWRITE_MODE) + // scalastyle:off caselocale + .map(mode => PartitionOverwriteMode.withName(mode.toUpperCase)) + // scalastyle:on caselocale + .getOrElse(conf.partitionOverwriteMode) + val enableDynamicOverwrite = partitionOverwriteMode == PartitionOverwriteMode.DYNAMIC + // This config only makes sense when we are overwriting a partitioned dataset with dynamic + // partition columns. + enableDynamicOverwrite && mode == SaveMode.Overwrite && + staticPartitions.size < partitionColumns.length + } + + override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { + // Most formats don't do well with duplicate columns, so lets not allow that + SchemaUtils.checkColumnNameDuplication( + outputColumnNames, + s"when inserting into $outputPath", + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(options) + val fs = outputPath.getFileSystem(hadoopConf) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + + val partitionsTrackedByCatalog = sparkSession.sessionState.conf.manageFilesourcePartitions && + catalogTable.isDefined && + catalogTable.get.partitionColumnNames.nonEmpty && + catalogTable.get.tracksPartitionsInCatalog + + var initialMatchingPartitions: Seq[TablePartitionSpec] = Nil + var customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty + var matchingPartitions: Seq[CatalogTablePartition] = Seq.empty + + // When partitions are tracked by the catalog, compute all custom partition locations that + // may be relevant to the insertion job. + if (partitionsTrackedByCatalog) { + matchingPartitions = sparkSession.sessionState.catalog + .listPartitions(catalogTable.get.identifier, Some(staticPartitions)) + initialMatchingPartitions = matchingPartitions.map(_.spec) + customPartitionLocations = + getCustomPartitionLocations(fs, catalogTable.get, qualifiedOutputPath, matchingPartitions) + } + + val jobId = java.util.UUID.randomUUID().toString + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = jobId, + outputPath = outputPath.toString, + dynamicPartitionOverwrite = dynamicPartitionOverwrite) + + val doInsertion = if (mode == SaveMode.Append) { + true + } else { + val pathExists = fs.exists(qualifiedOutputPath) + (mode, pathExists) match { + case (SaveMode.ErrorIfExists, true) => + throw QueryCompilationErrors.outputPathAlreadyExistsError(qualifiedOutputPath) + case (SaveMode.Overwrite, true) => + if (ifPartitionNotExists && matchingPartitions.nonEmpty) { + false + } else if (dynamicPartitionOverwrite) { + // For dynamic partition overwrite, do not delete partition directories ahead. + true + } else { + deleteMatchingPartitions(fs, qualifiedOutputPath, customPartitionLocations, committer) + true + } + case (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => + true + case (SaveMode.Ignore, exists) => + !exists + case (s, exists) => + throw QueryExecutionErrors.saveModeUnsupportedError(s, exists) + } + } + + if (doInsertion) { + + def refreshUpdatedPartitions(updatedPartitionPaths: Set[String]): Unit = { + val updatedPartitions = updatedPartitionPaths.map(PartitioningUtils.parsePathFragment) + if (partitionsTrackedByCatalog) { + val newPartitions = updatedPartitions -- initialMatchingPartitions + if (newPartitions.nonEmpty) { + AlterTableAddPartitionCommand( + catalogTable.get.identifier, + newPartitions.toSeq.map(p => (p, None)), + ifNotExists = true).run(sparkSession) + } + // For dynamic partition overwrite, we never remove partitions but only update existing + // ones. + if (mode == SaveMode.Overwrite && !dynamicPartitionOverwrite) { + val deletedPartitions = initialMatchingPartitions.toSet -- updatedPartitions + if (deletedPartitions.nonEmpty) { + AlterTableDropPartitionCommand( + catalogTable.get.identifier, + deletedPartitions.toSeq, + ifExists = true, + purge = false, + retainData = true /* already deleted */ ).run(sparkSession) + } + } + } + } + + // For dynamic partition overwrite, FileOutputCommitter's output path is staging path, files + // will be renamed from staging path to final output path during commit job + val committerOutputPath = if (dynamicPartitionOverwrite) { + FileCommitProtocol + .getStagingDir(outputPath.toString, jobId) + .makeQualified(fs.getUri, fs.getWorkingDirectory) + } else { + qualifiedOutputPath + } + + val updatedPartitionPaths = + FileFormatWriter.write( + sparkSession = sparkSession, + plan = child, + fileFormat = fileFormat, + committer = committer, + outputSpec = FileFormatWriter.OutputSpec( + committerOutputPath.toString, + customPartitionLocations, + outputColumns), + hadoopConf = hadoopConf, + partitionColumns = partitionColumns, + bucketSpec = bucketSpec, + statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), + options = options, + numStaticPartitionCols = staticPartitions.size + ) + + // update metastore partition metadata + if ( + updatedPartitionPaths.isEmpty && staticPartitions.nonEmpty + && partitionColumns.length == staticPartitions.size + ) { + // Avoid empty static partition can't loaded to datasource table. + val staticPathFragment = + PartitioningUtils.getPathFragment(staticPartitions, partitionColumns) + refreshUpdatedPartitions(Set(staticPathFragment)) + } else { + refreshUpdatedPartitions(updatedPartitionPaths) + } + + // refresh cached files in FileIndex + fileIndex.foreach(_.refresh()) + // refresh data cache if table is cached + sparkSession.sharedState.cacheManager.recacheByPath(sparkSession, outputPath, fs) + + if (catalogTable.nonEmpty) { + CommandUtils.updateTableStats(sparkSession, catalogTable.get) + } + + } else { + logInfo("Skipping insertion into a relation that already exists.") + } + + Seq.empty[Row] + } + + /** + * Deletes all partition files that match the specified static prefix. Partitions with custom + * locations are also cleared based on the custom locations map given to this class. + */ + private def deleteMatchingPartitions( + fs: FileSystem, + qualifiedOutputPath: Path, + customPartitionLocations: Map[TablePartitionSpec, String], + committer: FileCommitProtocol): Unit = { + val staticPartitionPrefix = if (staticPartitions.nonEmpty) { + "/" + partitionColumns + .flatMap(p => staticPartitions.get(p.name).map(getPartitionPathString(p.name, _))) + .mkString("/") + } else { + "" + } + // first clear the path determined by the static partition keys (e.g. /table/foo=1) + val staticPrefixPath = qualifiedOutputPath.suffix(staticPartitionPrefix) + if (fs.exists(staticPrefixPath) && !committer.deleteWithJob(fs, staticPrefixPath, true)) { + throw QueryExecutionErrors.cannotClearOutputDirectoryError(staticPrefixPath) + } + // now clear all custom partition locations (e.g. /custom/dir/where/foo=2/bar=4) + for ((spec, customLoc) <- customPartitionLocations) { + assert( + (staticPartitions.toSet -- spec).isEmpty, + "Custom partition location did not match static partitioning keys") + val path = new Path(customLoc) + if (fs.exists(path) && !committer.deleteWithJob(fs, path, true)) { + throw QueryExecutionErrors.cannotClearPartitionDirectoryError(path) + } + } + } + + /** + * Given a set of input partitions, returns those that have locations that differ from the Hive + * default (e.g. /k1=v1/k2=v2). These partitions were manually assigned locations by the user. + * + * @return + * a mapping from partition specs to their custom locations + */ + private def getCustomPartitionLocations( + fs: FileSystem, + table: CatalogTable, + qualifiedOutputPath: Path, + partitions: Seq[CatalogTablePartition]): Map[TablePartitionSpec, String] = { + partitions.flatMap { + p => + val defaultLocation = qualifiedOutputPath + .suffix("/" + PartitioningUtils.getPathFragment(p.spec, table.partitionSchema)) + .toString + val catalogLocation = + new Path(p.location).makeQualified(fs.getUri, fs.getWorkingDirectory).toString + if (catalogLocation != defaultLocation) { + Some(p.spec -> catalogLocation) + } else { + None + } + }.toMap + } + + override protected def withNewChildInternal( + newChild: LogicalPlan): InsertIntoHadoopFsRelationCommand = copy(query = newChild) +} + +// scalastyle:on line.size.limit diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala index 0fe83094a780..0a81d1b28922 100644 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala @@ -16,13 +16,12 @@ */ package org.apache.spark.sql.execution.datasources.v2 -import io.glutenproject.GlutenConfig - import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning import org.apache.spark.sql.catalyst.util.InternalRowSet +import org.apache.spark.sql.connector.catalog.Table import org.apache.spark.sql.connector.expressions.aggregate.Aggregation import org.apache.spark.sql.connector.read.{HasPartitionKey, InputPartition, Scan, SupportsRuntimeFiltering} import org.apache.spark.sql.execution.datasources.DataSourceStrategy @@ -34,14 +33,13 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class BatchScanExecShim( output: Seq[AttributeReference], @transient scan: Scan, - runtimeFilters: Seq[Expression]) + runtimeFilters: Seq[Expression], + @transient table: Table) extends BatchScanExec(output, scan, runtimeFilters) { // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. @transient override lazy val metrics: Map[String, SQLMetric] = Map() - override def supportsColumnar(): Boolean = GlutenConfig.getConf.enableColumnarIterator - override def doExecuteColumnar(): RDD[ColumnarBatch] = { throw new UnsupportedOperationException("Need to implement this method") } @@ -120,4 +118,8 @@ class BatchScanExecShim( case _ => None } } + + final override protected def otherCopyArgs: Seq[AnyRef] = { + output :: scan :: runtimeFilters :: Nil + } } diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala deleted file mode 100644 index 6536f8081474..000000000000 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScan.scala +++ /dev/null @@ -1,43 +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.datasources.v2.velox - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.Expression -import org.apache.spark.sql.connector.read.PartitionReaderFactory -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScan -import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class DwrfScan( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - readDataSchema: StructType, - readPartitionSchema: StructType, - pushedFilters: Array[Filter], - options: CaseInsensitiveStringMap, - partitionFilters: Seq[Expression] = Seq.empty, - dataFilters: Seq[Expression] = Seq.empty) - extends FileScan { - override def createReaderFactory(): PartitionReaderFactory = { - null - } - - override def dataSchema: StructType = readDataSchema -} diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala deleted file mode 100644 index 475b18b68531..000000000000 --- a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/datasources/v2/velox/DwrfScanBuilder.scala +++ /dev/null @@ -1,44 +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.datasources.v2.velox - -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan -import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex -import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder -import org.apache.spark.sql.types.StructType -import org.apache.spark.sql.util.CaseInsensitiveStringMap - -case class DwrfScanBuilder( - sparkSession: SparkSession, - fileIndex: PartitioningAwareFileIndex, - schema: StructType, - dataSchema: StructType, - options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { - - override def build(): Scan = { - DwrfScan( - sparkSession, - fileIndex, - readDataSchema(), - readPartitionSchema(), - pushedDataFilters, - options) - } - -} diff --git a/shims/spark33/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala new file mode 100644 index 000000000000..08ba7680ca70 --- /dev/null +++ b/shims/spark33/src/main/scala/org/apache/spark/sql/execution/stat/StatFunctions.scala @@ -0,0 +1,364 @@ +/* + * 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.stat + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Expression, GenericInternalRow, GetArrayItem, Literal, TryCast} +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.util.{GenericArrayData, QuantileSummaries} +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.functions.count +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +import java.util.Locale + +/** + * This file is copied from Spark + * + * The df.describe() and df.summary() issues are fixed by + * https://github.com/apache/spark/pull/40914. We picked it into Gluten to fix the describe and + * summary issue. And this file can be removed after upgrading spark version to 3.4 or higher + * version. + */ +object StatFunctions extends Logging { + + /** + * Calculates the approximate quantiles of multiple numerical columns of a DataFrame in one pass. + * + * The result of this algorithm has the following deterministic bound: If the DataFrame has N + * elements and if we request the quantile at probability `p` up to error `err`, then the + * algorithm will return a sample `x` from the DataFrame so that the *exact* rank of `x` is close + * to (p * N). More precisely, + * + * floor((p - err) * N) <= rank(x) <= ceil((p + err) * N). + * + * This method implements a variation of the Greenwald-Khanna algorithm (with some speed + * optimizations). The algorithm was first present in Space-efficient Online Computation of Quantile + * Summaries by Greenwald and Khanna. + * + * @param df + * the dataframe + * @param cols + * numerical columns of the dataframe + * @param probabilities + * a list of quantile probabilities Each number must belong to [0, 1]. For example 0 is the + * minimum, 0.5 is the median, 1 is the maximum. + * @param relativeError + * The relative target precision to achieve (greater than or equal 0). If set to zero, the exact + * quantiles are computed, which could be very expensive. Note that values greater than 1 are + * accepted but give the same result as 1. + * @return + * for each column, returns the requested approximations + * @note + * null and NaN values will be ignored in numerical columns before calculation. For a column + * only containing null or NaN values, an empty array is returned. + */ + def multipleApproxQuantiles( + df: DataFrame, + cols: Seq[String], + probabilities: Seq[Double], + relativeError: Double): Seq[Seq[Double]] = { + require(relativeError >= 0, s"Relative Error must be non-negative but got $relativeError") + val columns: Seq[Column] = cols.map { + colName => + val field = df.resolve(colName) + require( + field.dataType.isInstanceOf[NumericType], + s"Quantile calculation for column $colName with data type ${field.dataType}" + + " is not supported.") + Column(Cast(Column(colName).expr, DoubleType)) + } + val emptySummaries = Array.fill(cols.size)( + new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, relativeError)) + + // Note that it works more or less by accident as `rdd.aggregate` is not a pure function: + // this function returns the same array as given in the input (because `aggregate` reuses + // the same argument). + def apply(summaries: Array[QuantileSummaries], row: Row): Array[QuantileSummaries] = { + var i = 0 + while (i < summaries.length) { + if (!row.isNullAt(i)) { + val v = row.getDouble(i) + if (!v.isNaN) summaries(i) = summaries(i).insert(v) + } + i += 1 + } + summaries + } + + def merge( + sum1: Array[QuantileSummaries], + sum2: Array[QuantileSummaries]): Array[QuantileSummaries] = { + sum1.zip(sum2).map { case (s1, s2) => s1.compress().merge(s2.compress()) } + } + + val summaries = df.select(columns: _*).rdd.treeAggregate(emptySummaries)(apply, merge) + + summaries.map { + summary => + summary.query(probabilities) match { + case Some(q) => q + case None => Seq() + } + } + } + + /** Calculate the Pearson Correlation Coefficient for the given columns */ + def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = { + val counts = collectStatisticalData(df, cols, "correlation") + counts.Ck / math.sqrt(counts.MkX * counts.MkY) + } + + /** Helper class to simplify tracking and merging counts. */ + private class CovarianceCounter extends Serializable { + var xAvg = 0.0 // the mean of all examples seen so far in col1 + var yAvg = 0.0 // the mean of all examples seen so far in col2 + var Ck = 0.0 // the co-moment after k examples + var MkX = 0.0 // sum of squares of differences from the (current) mean for col1 + var MkY = 0.0 // sum of squares of differences from the (current) mean for col2 + var count = 0L // count of observed examples + + // add an example to the calculation + def add(x: Double, y: Double): this.type = { + val deltaX = x - xAvg + val deltaY = y - yAvg + count += 1 + xAvg += deltaX / count + yAvg += deltaY / count + Ck += deltaX * (y - yAvg) + MkX += deltaX * (x - xAvg) + MkY += deltaY * (y - yAvg) + this + } + + // merge counters from other partitions. Formula can be found at: + // http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance + def merge(other: CovarianceCounter): this.type = { + if (other.count > 0) { + val totalCount = count + other.count + val deltaX = xAvg - other.xAvg + val deltaY = yAvg - other.yAvg + Ck += other.Ck + deltaX * deltaY * count / totalCount * other.count + xAvg = (xAvg * count + other.xAvg * other.count) / totalCount + yAvg = (yAvg * count + other.yAvg * other.count) / totalCount + MkX += other.MkX + deltaX * deltaX * count / totalCount * other.count + MkY += other.MkY + deltaY * deltaY * count / totalCount * other.count + count = totalCount + } + this + } + + // return the sample covariance for the observed examples + def cov: Double = Ck / (count - 1) + } + + private def collectStatisticalData( + df: DataFrame, + cols: Seq[String], + functionName: String): CovarianceCounter = { + require( + cols.length == 2, + s"Currently $functionName calculation is supported " + + "between two columns.") + cols.map(name => (name, df.resolve(name))).foreach { + case (name, data) => + require( + data.dataType.isInstanceOf[NumericType], + s"Currently $functionName calculation " + + s"for columns with dataType ${data.dataType.catalogString} not supported." + ) + } + val columns = cols.map(n => Column(Cast(Column(n).expr, DoubleType))) + df.select(columns: _*) + .queryExecution + .toRdd + .treeAggregate(new CovarianceCounter)( + seqOp = (counter, row) => { + counter.add(row.getDouble(0), row.getDouble(1)) + }, + combOp = (baseCounter, other) => { + baseCounter.merge(other) + }) + } + + /** + * Calculate the covariance of two numerical columns of a DataFrame. + * + * @param df + * The DataFrame + * @param cols + * the column names + * @return + * the covariance of the two columns. + */ + def calculateCov(df: DataFrame, cols: Seq[String]): Double = { + val counts = collectStatisticalData(df, cols, "covariance") + counts.cov + } + + /** Generate a table of frequencies for the elements of two columns. */ + def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = { + val tableName = s"${col1}_$col2" + val counts = df.groupBy(col1, col2).agg(count("*")).take(1e6.toInt) + if (counts.length == 1e6.toInt) { + logWarning( + "The maximum limit of 1e6 pairs have been collected, which may not be all of " + + "the pairs. Please try reducing the amount of distinct items in your columns.") + } + + def cleanElement(element: Any): String = { + if (element == null) "null" else element.toString + } + + // get the distinct sorted values of column 2, so that we can make them the column names + val distinctCol2: Map[Any, Int] = + counts.map(e => cleanElement(e.get(1))).distinct.sorted.zipWithIndex.toMap + val columnSize = distinctCol2.size + require( + columnSize < 1e4, + s"The number of distinct values for $col2, can't " + + s"exceed 1e4. Currently $columnSize") + val table = counts + .groupBy(_.get(0)) + .map { + case (col1Item, rows) => + val countsRow = new GenericInternalRow(columnSize + 1) + rows.foreach { + (row: Row) => + // row.get(0) is column 1 + // row.get(1) is column 2 + // row.get(2) is the frequency + val columnIndex = distinctCol2(cleanElement(row.get(1))) + countsRow.setLong(columnIndex + 1, row.getLong(2)) + } + // the value of col1 is the first value, the rest are the counts + countsRow.update(0, UTF8String.fromString(cleanElement(col1Item))) + countsRow + } + .toSeq + + // Back ticks can't exist in DataFrame column names, therefore drop them. To be able to accept + // special keywords and `.`, wrap the column names in ``. + def cleanColumnName(name: String): String = { + name.replace("`", "") + } + + // In the map, the column names (._1) are not ordered by the index (._2). This was the bug in + // SPARK-8681. We need to explicitly sort by the column index and assign the column names. + val headerNames = distinctCol2.toSeq.sortBy(_._2).map { + r => StructField(cleanColumnName(r._1.toString), LongType) + } + val schema = StructType(StructField(tableName, StringType) +: headerNames) + + Dataset.ofRows(df.sparkSession, LocalRelation(schema.toAttributes, table)).na.fill(0.0) + } + + /** Calculate selected summary statistics for a dataset */ + def summary(ds: Dataset[_], statistics: Seq[String]): DataFrame = { + + val defaultStatistics = Seq("count", "mean", "stddev", "min", "25%", "50%", "75%", "max") + val selectedStatistics = if (statistics.nonEmpty) statistics else defaultStatistics + + val percentiles = selectedStatistics.filter(a => a.endsWith("%")).map { + p => + try { + p.stripSuffix("%").toDouble / 100.0 + } catch { + case e: NumberFormatException => + throw QueryExecutionErrors.cannotParseStatisticAsPercentileError(p, e) + } + } + require(percentiles.forall(p => p >= 0 && p <= 1), "Percentiles must be in the range [0, 1]") + + def castAsDoubleIfNecessary(e: Expression): Expression = if (e.dataType == StringType) { + TryCast(e, DoubleType) + } else { + e + } + + var percentileIndex = 0 + val statisticFns = selectedStatistics.map { + stats => + if (stats.endsWith("%")) { + val index = percentileIndex + percentileIndex += 1 + (child: Expression) => + GetArrayItem( + new ApproximatePercentile( + castAsDoubleIfNecessary(child), + Literal(new GenericArrayData(percentiles), ArrayType(DoubleType, false))) + .toAggregateExpression(), + Literal(index) + ) + } else { + stats.toLowerCase(Locale.ROOT) match { + case "count" => (child: Expression) => Count(child).toAggregateExpression() + case "count_distinct" => + (child: Expression) => Count(child).toAggregateExpression(isDistinct = true) + case "approx_count_distinct" => + (child: Expression) => HyperLogLogPlusPlus(child).toAggregateExpression() + case "mean" => + (child: Expression) => Average(castAsDoubleIfNecessary(child)).toAggregateExpression() + case "stddev" => + (child: Expression) => + StddevSamp(castAsDoubleIfNecessary(child)).toAggregateExpression() + case "min" => (child: Expression) => Min(child).toAggregateExpression() + case "max" => (child: Expression) => Max(child).toAggregateExpression() + case _ => throw QueryExecutionErrors.statisticNotRecognizedError(stats) + } + } + } + + val selectedCols = ds.logicalPlan.output + .filter(a => a.dataType.isInstanceOf[NumericType] || a.dataType.isInstanceOf[StringType]) + + val aggExprs = statisticFns.flatMap { + func => selectedCols.map(c => Column(Cast(func(c), StringType)).as(c.name)) + } + + // If there is no selected columns, we don't need to run this aggregate, so make it a lazy val. + lazy val aggResult = ds.select(aggExprs: _*).queryExecution.toRdd.map(_.copy()).collect().head + + // We will have one row for each selected statistic in the result. + val result = Array.fill[InternalRow](selectedStatistics.length) { + // each row has the statistic name, and statistic values of each selected column. + new GenericInternalRow(selectedCols.length + 1) + } + + var rowIndex = 0 + while (rowIndex < result.length) { + val statsName = selectedStatistics(rowIndex) + result(rowIndex).update(0, UTF8String.fromString(statsName)) + for (colIndex <- selectedCols.indices) { + val statsValue = aggResult.getUTF8String(rowIndex * selectedCols.length + colIndex) + result(rowIndex).update(colIndex + 1, statsValue) + } + rowIndex += 1 + } + + // All columns are string type + val output = AttributeReference("summary", StringType)() +: + selectedCols.map(c => AttributeReference(c.name, StringType)()) + + Dataset.ofRows(ds.sparkSession, LocalRelation(output, result)) + } +} diff --git a/shims/spark34/pom.xml b/shims/spark34/pom.xml new file mode 100644 index 000000000000..5b10f8761bbc --- /dev/null +++ b/shims/spark34/pom.xml @@ -0,0 +1,125 @@ + + + + 4.0.0 + + + io.glutenproject + spark-sql-columnar-shims + 1.1.0-SNAPSHOT + ../pom.xml + + + spark-sql-columnar-shims-spark34 + Gluten Shims for Spark 3.4 + jar + + + + io.glutenproject + ${project.prefix}-shims-common + ${project.version} + compile + + + org.apache.spark + spark-sql_${scala.binary.version} + provided + true + + + org.apache.spark + spark-catalyst_2.12 + provided + true + + + org.apache.spark + spark-core_2.12 + provided + true + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided + + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.apache.spark + spark-core_${scala.binary.version} + test-jar + + + org.apache.spark + spark-sql_${scala.binary.version} + test-jar + + + org.apache.spark + spark-catalyst_${scala.binary.version} + test-jar + + + org.apache.spark + spark-hive_${scala.binary.version} + provided + + + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.scalastyle + scalastyle-maven-plugin + + + com.diffplug.spotless + spotless-maven-plugin + + + net.alchim31.maven + scala-maven-plugin + + + -Wconf:cat=deprecation:silent + + + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-compiler-plugin + + + + diff --git a/shims/spark34/src/main/java/org/apache/spark/sql/execution/joins/HashJoin.scala.deprecated b/shims/spark34/src/main/java/org/apache/spark/sql/execution/joins/HashJoin.scala.deprecated new file mode 100644 index 000000000000..3fc823730614 --- /dev/null +++ b/shims/spark34/src/main/java/org/apache/spark/sql/execution/joins/HashJoin.scala.deprecated @@ -0,0 +1,774 @@ +/* + * 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.joins + +import org.apache.spark.sql.catalyst.{InternalRow, SQLConfHelper} +import org.apache.spark.sql.catalyst.analysis.CastSupport +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{CodegenSupport, ExplainUtils, RowIterator} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.{BooleanType, IntegralType, LongType} + +/** + * @param relationTerm variable name for HashedRelation + * @param keyIsUnique indicate whether keys of HashedRelation known to be unique in code-gen time + * @param isEmpty indicate whether it known to be EmptyHashedRelation in code-gen time + */ +private[joins] case class HashedRelationInfo( + relationTerm: String, + keyIsUnique: Boolean, + isEmpty: Boolean) + +trait HashJoin extends JoinCodegenSupport { + def buildSide: BuildSide + + override def simpleStringWithNodeId(): String = { + val opId = ExplainUtils.getOpId(this) + s"$nodeName $joinType ${buildSide} ($opId)".trim + } + + override def output: Seq[Attribute] = { + joinType match { + case _: InnerLike => + left.output ++ right.output + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case j: ExistenceJoin => + left.output :+ j.exists + case LeftExistence(_) => + left.output + case x => + throw new IllegalArgumentException(s"HashJoin should not take $x as the JoinType") + } + } + + override def outputPartitioning: Partitioning = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputPartitioning + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } + + /** + * Handle the special cases for LeftOuter/LeftSemi with BuildLeft and RightOuter with BuildRight. + */ + override def outputOrdering: Seq[SortOrder] = buildSide match { + case BuildLeft => + joinType match { + case _: InnerLike | RightOuter => right.outputOrdering + case LeftOuter => left.outputOrdering + case LeftSemi => left.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building left side") + } + case BuildRight => + joinType match { + case _: InnerLike | LeftOuter | LeftSemi | LeftAnti | _: ExistenceJoin => + left.outputOrdering + case RightOuter => right.outputOrdering + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType with building right side") + } + } + + protected lazy val (buildPlan, streamedPlan) = buildSide match { + case BuildLeft => (left, right) + case BuildRight => (right, left) + } + + protected lazy val (buildKeys, streamedKeys) = { + require(leftKeys.length == rightKeys.length && + leftKeys.map(_.dataType) + .zip(rightKeys.map(_.dataType)) + .forall(types => types._1.sameType(types._2)), + "Join keys from two sides should have same length and types") + buildSide match { + case BuildLeft => (leftKeys, rightKeys) + case BuildRight => (rightKeys, leftKeys) + } + } + + @transient protected lazy val (buildOutput, streamedOutput) = { + buildSide match { + case BuildLeft => (left.output, right.output) + case BuildRight => (right.output, left.output) + } + } + + @transient protected lazy val buildBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(buildKeys), buildOutput) + + @transient protected lazy val streamedBoundKeys = + bindReferences(HashJoin.rewriteKeyExpr(streamedKeys), streamedOutput) + + protected def buildSideKeyGenerator(): Projection = + UnsafeProjection.create(buildBoundKeys) + + protected def streamSideKeyGenerator(): UnsafeProjection = + UnsafeProjection.create(streamedBoundKeys) + + @transient protected[this] lazy val boundCondition = if (condition.isDefined) { + if (joinType == FullOuter && buildSide == BuildLeft) { + // Put join left side before right side. This is to be consistent with + // `ShuffledHashJoinExec.fullOuterJoin`. + Predicate.create(condition.get, buildPlan.output ++ streamedPlan.output).eval _ + } else { + Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + } + } else { + (r: InternalRow) => true + } + + protected def createResultProjection(): (InternalRow) => InternalRow = joinType match { + case LeftExistence(_) => + UnsafeProjection.create(output, output) + case _ => + // Always put the stream side on left to simplify implementation + // both of left and right side could be null + UnsafeProjection.create( + output, (streamedPlan.output ++ buildPlan.output).map(_.withNullability(true))) + } + + private def innerJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinRow = new JoinedRow + val joinKeys = streamSideKeyGenerator() + + if (hashedRelation == EmptyHashedRelation) { + Iterator.empty + } else if (hashedRelation.keyIsUnique) { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matched = hashedRelation.getValue(joinKeys(srow)) + if (matched != null) { + Some(joinRow.withRight(matched)).filter(boundCondition) + } else { + None + } + } + } else { + streamIter.flatMap { srow => + joinRow.withLeft(srow) + val matches = hashedRelation.get(joinKeys(srow)) + if (matches != null) { + matches.map(joinRow.withRight).filter(boundCondition) + } else { + Seq.empty + } + } + } + } + + private def outerJoin( + streamedIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinedRow = new JoinedRow() + val keyGenerator = streamSideKeyGenerator() + val nullRow = new GenericInternalRow(buildPlan.output.length) + + if (hashedRelation.keyIsUnique) { + streamedIter.map { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val matched = hashedRelation.getValue(rowKey) + if (matched != null && boundCondition(joinedRow.withRight(matched))) { + joinedRow + } else { + joinedRow.withRight(nullRow) + } + } + } else { + streamedIter.flatMap { currentRow => + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + val buildIter = hashedRelation.get(rowKey) + new RowIterator { + private var found = false + override def advanceNext(): Boolean = { + while (buildIter != null && buildIter.hasNext) { + val nextBuildRow = buildIter.next() + if (boundCondition(joinedRow.withRight(nextBuildRow))) { + found = true + return true + } + } + if (!found) { + joinedRow.withRight(nullRow) + found = true + return true + } + false + } + override def getRow: InternalRow = joinedRow + }.toScala + } + } + } + + private def semiJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinKeys = streamSideKeyGenerator() + val joinedRow = new JoinedRow + + if (hashedRelation == EmptyHashedRelation) { + Iterator.empty + } else if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + } + } + } + + private def existenceJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + val joinKeys = streamSideKeyGenerator() + val result = new GenericInternalRow(Array[Any](null)) + val joinedRow = new JoinedRow + + if (hashedRelation.keyIsUnique) { + streamIter.map { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + val exists = !key.anyNull && matched != null && + (condition.isEmpty || boundCondition(joinedRow(current, matched))) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } else { + streamIter.map { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + val exists = !key.anyNull && buildIter != null && (condition.isEmpty || buildIter.exists { + (row: InternalRow) => boundCondition(joinedRow(current, row)) + }) + result.setBoolean(0, exists) + joinedRow(current, result) + } + } + } + + private def antiJoin( + streamIter: Iterator[InternalRow], + hashedRelation: HashedRelation): Iterator[InternalRow] = { + // If the right side is empty, AntiJoin simply returns the left side. + if (hashedRelation == EmptyHashedRelation) { + return streamIter + } + + val joinKeys = streamSideKeyGenerator() + val joinedRow = new JoinedRow + + if (hashedRelation.keyIsUnique) { + streamIter.filter { current => + val key = joinKeys(current) + lazy val matched = hashedRelation.getValue(key) + key.anyNull || matched == null || + (condition.isDefined && !boundCondition(joinedRow(current, matched))) + } + } else { + streamIter.filter { current => + val key = joinKeys(current) + lazy val buildIter = hashedRelation.get(key) + key.anyNull || buildIter == null || (condition.isDefined && !buildIter.exists { + row => boundCondition(joinedRow(current, row)) + }) + } + } + } + + protected def join( + streamedIter: Iterator[InternalRow], + hashed: HashedRelation, + numOutputRows: SQLMetric): Iterator[InternalRow] = { + + val joinedIter = joinType match { + case _: InnerLike => + innerJoin(streamedIter, hashed) + case LeftOuter | RightOuter => + outerJoin(streamedIter, hashed) + case LeftSemi => + semiJoin(streamedIter, hashed) + case LeftAnti => + antiJoin(streamedIter, hashed) + case _: ExistenceJoin => + existenceJoin(streamedIter, hashed) + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType") + } + + val resultProj = createResultProjection + joinedIter.map { r => + numOutputRows += 1 + resultProj(r) + } + } + + override def doProduce(ctx: CodegenContext): String = { + streamedPlan.asInstanceOf[CodegenSupport].produce(ctx, this) + } + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { + joinType match { + case _: InnerLike => codegenInner(ctx, input) + case LeftOuter | RightOuter => codegenOuter(ctx, input) + case LeftSemi => codegenSemi(ctx, input) + case LeftAnti => codegenAnti(ctx, input) + case _: ExistenceJoin => codegenExistence(ctx, input) + case x => + throw new IllegalArgumentException( + s"HashJoin should not take $x as the JoinType") + } + } + + /** + * Returns the code for generating join key for stream side, and expression of whether the key + * has any null in it or not. + */ + protected def genStreamSideJoinKey( + ctx: CodegenContext, + input: Seq[ExprCode]): (ExprCode, String) = { + ctx.currentVars = input + if (streamedBoundKeys.length == 1 && streamedBoundKeys.head.dataType == LongType) { + // generate the join key as Long + val ev = streamedBoundKeys.head.genCode(ctx) + (ev, ev.isNull) + } else { + // generate the join key as UnsafeRow + val ev = GenerateUnsafeProjection.createCode(ctx, streamedBoundKeys) + (ev, s"${ev.value}.anyNull()") + } + } + + /** + * Generates the code for Inner join. + */ + protected def codegenInner(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val HashedRelationInfo(relationTerm, keyIsUnique, isEmptyHashedRelation) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, buildVars) = getJoinCondition(ctx, input, streamedPlan, buildPlan) + val numOutput = metricTerm(ctx, "numOutputRows") + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (isEmptyHashedRelation) { + """ + |// If HashedRelation is empty, hash inner join simply returns nothing. + """.stripMargin + } else if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? + | null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | while ($matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left or right outer join. + */ + protected def codegenOuter(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val HashedRelationInfo(relationTerm, keyIsUnique, _) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val matched = ctx.freshName("matched") + val buildVars = genOneSideJoinVars(ctx, matched, buildPlan, setDefaultValue = true) + val numOutput = metricTerm(ctx, "numOutputRows") + + // filter the output via condition + val conditionPassed = ctx.freshName("conditionPassed") + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |boolean $conditionPassed = true; + |${eval.trim} + |if ($matched != null) { + | ${ev.code} + | $conditionPassed = !${ev.isNull} && ${ev.value}; + |} + """.stripMargin + } else { + s"final boolean $conditionPassed = true;" + } + + val resultVars = buildSide match { + case BuildLeft => buildVars ++ input + case BuildRight => input ++ buildVars + } + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |${checkCondition.trim} + |if (!$conditionPassed) { + | $matched = null; + | // reset the variables those are already evaluated. + | ${buildVars.filter(_.code.isEmpty).map(v => s"${v.isNull} = true;").mkString("\n")} + |} + |$numOutput.add(1); + |${consume(ctx, resultVars)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $found = false; + |// the last iteration of this loop is to emit an empty row if there is no matched rows. + |while ($matches != null && $matches.hasNext() || !$found) { + | UnsafeRow $matched = $matches != null && $matches.hasNext() ? + | (UnsafeRow) $matches.next() : null; + | ${checkCondition.trim} + | if ($conditionPassed) { + | $found = true; + | $numOutput.add(1); + | ${consume(ctx, resultVars)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for left semi join. + */ + protected def codegenSemi(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val HashedRelationInfo(relationTerm, keyIsUnique, isEmptyHashedRelation) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input, streamedPlan, buildPlan) + val numOutput = metricTerm(ctx, "numOutputRows") + + if (isEmptyHashedRelation) { + """ + |// If HashedRelation is empty, hash semi join simply returns nothing. + """.stripMargin + } else if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |if ($matched != null) { + | $checkCondition { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |if ($matches != null) { + | boolean $found = false; + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | if ($found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + | } + |} + """.stripMargin + } + } + + /** + * Generates the code for anti join. + */ + protected def codegenAnti(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val HashedRelationInfo(relationTerm, keyIsUnique, isEmptyHashedRelation) = prepareRelation(ctx) + val numOutput = metricTerm(ctx, "numOutputRows") + if (isEmptyHashedRelation) { + return s""" + |// If HashedRelation is empty, hash anti join simply returns the stream side. + |$numOutput.add(1); + |${consume(ctx, input)} + """.stripMargin + } + + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val (matched, checkCondition, _) = getJoinCondition(ctx, input, streamedPlan, buildPlan) + + if (keyIsUnique) { + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | UnsafeRow $matched = (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + | if ($matched != null) { + | // Evaluate the condition. + | $checkCondition { + | $found = true; + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + val found = ctx.freshName("found") + s""" + |boolean $found = false; + |// generate join key for stream side + |${keyEv.code} + |// Check if the key has nulls. + |if (!($anyNull)) { + | // Check if the HashedRelation exists. + | $iteratorCls $matches = ($iteratorCls)$relationTerm.get(${keyEv.value}); + | if ($matches != null) { + | // Evaluate the condition. + | while (!$found && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition { + | $found = true; + | } + | } + | } + |} + |if (!$found) { + | $numOutput.add(1); + | ${consume(ctx, input)} + |} + """.stripMargin + } + } + + /** + * Generates the code for existence join. + */ + protected def codegenExistence(ctx: CodegenContext, input: Seq[ExprCode]): String = { + val HashedRelationInfo(relationTerm, keyIsUnique, _) = prepareRelation(ctx) + val (keyEv, anyNull) = genStreamSideJoinKey(ctx, input) + val numOutput = metricTerm(ctx, "numOutputRows") + val existsVar = ctx.freshName("exists") + + val matched = ctx.freshName("matched") + val buildVars = genOneSideJoinVars(ctx, matched, buildPlan, setDefaultValue = false) + val checkCondition = if (condition.isDefined) { + val expr = condition.get + // evaluate the variables from build side that used by condition + val eval = evaluateRequiredVariables(buildPlan.output, buildVars, expr.references) + // filter the output via condition + ctx.currentVars = input ++ buildVars + val ev = + BindReferences.bindReference(expr, streamedPlan.output ++ buildPlan.output).genCode(ctx) + s""" + |$eval + |${ev.code} + |$existsVar = !${ev.isNull} && ${ev.value}; + """.stripMargin + } else { + s"$existsVar = true;" + } + + val resultVar = input ++ Seq(ExprCode.forNonNullValue( + JavaCode.variable(existsVar, BooleanType))) + + if (keyIsUnique) { + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashedRelation + |UnsafeRow $matched = $anyNull ? null: (UnsafeRow)$relationTerm.getValue(${keyEv.value}); + |boolean $existsVar = false; + |if ($matched != null) { + | $checkCondition + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } else { + val matches = ctx.freshName("matches") + val iteratorCls = classOf[Iterator[UnsafeRow]].getName + s""" + |// generate join key for stream side + |${keyEv.code} + |// find matches from HashRelation + |$iteratorCls $matches = $anyNull ? null : ($iteratorCls)$relationTerm.get(${keyEv.value}); + |boolean $existsVar = false; + |if ($matches != null) { + | while (!$existsVar && $matches.hasNext()) { + | UnsafeRow $matched = (UnsafeRow) $matches.next(); + | $checkCondition + | } + |} + |$numOutput.add(1); + |${consume(ctx, resultVar)} + """.stripMargin + } + } + + protected def prepareRelation(ctx: CodegenContext): HashedRelationInfo +} + +object HashJoin extends CastSupport with SQLConfHelper { + + private def canRewriteAsLongType(keys: Seq[Expression]): Boolean = { + // TODO: support BooleanType, DateType and TimestampType + keys.forall(_.dataType.isInstanceOf[IntegralType]) && + keys.map(_.dataType.defaultSize).sum <= 8 + } + + /** + * Try to rewrite the key as LongType so we can use getLong(), if they key can fit with a long. + * + * If not, returns the original expressions. + */ + def rewriteKeyExpr(keys: Seq[Expression]): Seq[Expression] = { + assert(keys.nonEmpty) + if (!canRewriteAsLongType(keys)) { + return keys + } + + var keyExpr: Expression = if (keys.head.dataType != LongType) { + cast(keys.head, LongType) + } else { + keys.head + } + keys.tail.foreach { e => + val bits = e.dataType.defaultSize * 8 + keyExpr = BitwiseOr(ShiftLeft(keyExpr, Literal(bits)), + BitwiseAnd(cast(e, LongType), Literal((1L << bits) - 1))) + } + keyExpr :: Nil + } + + /** + * Extract a given key which was previously packed in a long value using its index to + * determine the number of bits to shift + */ + def extractKeyExprAt(keys: Seq[Expression], index: Int): Expression = { + assert(canRewriteAsLongType(keys)) + // jump over keys that have a higher index value than the required key + if (keys.size == 1) { + assert(index == 0) + Cast( + child = BoundReference(0, LongType, nullable = false), + dataType = keys(index).dataType, + timeZoneId = Option(conf.sessionLocalTimeZone), + ansiEnabled = false) + } else { + val shiftedBits = + keys.slice(index + 1, keys.size).map(_.dataType.defaultSize * 8).sum + val mask = (1L << (keys(index).dataType.defaultSize * 8)) - 1 + // build the schema for unpacking the required key + val castChild = BitwiseAnd( + ShiftRightUnsigned(BoundReference(0, LongType, nullable = false), Literal(shiftedBits)), + Literal(mask)) + Cast( + child = castChild, + dataType = keys(index).dataType, + timeZoneId = Option(conf.sessionLocalTimeZone), + ansiEnabled = false) + } + } +} diff --git a/shims/spark34/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVectorShim.java b/shims/spark34/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVectorShim.java new file mode 100644 index 000000000000..1b77cb666c72 --- /dev/null +++ b/shims/spark34/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVectorShim.java @@ -0,0 +1,223 @@ +/* + * 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.vectorized; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.unsafe.types.UTF8String; + +import java.nio.ByteBuffer; + +/** + * because spark33 add new function abstract method 'putBooleans(int, byte)' in + * 'WritableColumnVector' And function getByteBuffer() + */ +public class WritableColumnVectorShim extends WritableColumnVector { + /** + * Sets up the common state and also handles creating the child columns if this is a nested type. + * + * @param capacity + * @param type + */ + protected WritableColumnVectorShim(int capacity, DataType type) { + super(capacity, type); + } + + @Override + public int getDictId(int rowId) { + return 0; + } + + @Override + protected void reserveInternal(int capacity) {} + + @Override + public void putNotNull(int rowId) {} + + @Override + public void putNull(int rowId) {} + + @Override + public void putNulls(int rowId, int count) {} + + @Override + public void putNotNulls(int rowId, int count) {} + + @Override + public void putBoolean(int rowId, boolean value) {} + + @Override + public void putBooleans(int rowId, int count, boolean value) {} + + @Override + public void putBooleans(int rowId, byte src) { + throw new UnsupportedOperationException("Unsupported function"); + } + + @Override + public void putByte(int rowId, byte value) {} + + @Override + public void putBytes(int rowId, int count, byte value) {} + + @Override + public void putBytes(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putShort(int rowId, short value) {} + + @Override + public void putShorts(int rowId, int count, short value) {} + + @Override + public void putShorts(int rowId, int count, short[] src, int srcIndex) {} + + @Override + public void putShorts(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putInt(int rowId, int value) {} + + @Override + public void putInts(int rowId, int count, int value) {} + + @Override + public void putInts(int rowId, int count, int[] src, int srcIndex) {} + + @Override + public void putInts(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putIntsLittleEndian(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putLong(int rowId, long value) {} + + @Override + public void putLongs(int rowId, int count, long value) {} + + @Override + public void putLongs(int rowId, int count, long[] src, int srcIndex) {} + + @Override + public void putLongs(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putFloat(int rowId, float value) {} + + @Override + public void putFloats(int rowId, int count, float value) {} + + @Override + public void putFloats(int rowId, int count, float[] src, int srcIndex) {} + + @Override + public void putFloats(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putFloatsLittleEndian(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putDouble(int rowId, double value) {} + + @Override + public void putDoubles(int rowId, int count, double value) {} + + @Override + public void putDoubles(int rowId, int count, double[] src, int srcIndex) {} + + @Override + public void putDoubles(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putDoublesLittleEndian(int rowId, int count, byte[] src, int srcIndex) {} + + @Override + public void putArray(int rowId, int offset, int length) {} + + @Override + public int putByteArray(int rowId, byte[] value, int offset, int count) { + return 0; + } + + @Override + protected UTF8String getBytesAsUTF8String(int rowId, int count) { + return null; + } + + @Override + public ByteBuffer getByteBuffer(int rowId, int count) { + throw new UnsupportedOperationException("Unsupported this function"); + } + + @Override + public int getArrayLength(int rowId) { + return 0; + } + + @Override + public int getArrayOffset(int rowId) { + return 0; + } + + @Override + protected WritableColumnVector reserveNewColumn(int capacity, DataType type) { + return null; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return false; + } + + @Override + public byte getByte(int rowId) { + return 0; + } + + @Override + public short getShort(int rowId) { + return 0; + } + + @Override + public int getInt(int rowId) { + return 0; + } + + @Override + public long getLong(int rowId) { + return 0; + } + + @Override + public float getFloat(int rowId) { + return 0; + } + + @Override + public double getDouble(int rowId) { + return 0; + } +} diff --git a/shims/spark34/src/main/resources/META-INF/services/io.glutenproject.sql.shims.SparkShimProvider b/shims/spark34/src/main/resources/META-INF/services/io.glutenproject.sql.shims.SparkShimProvider new file mode 100644 index 000000000000..1566cfff8d0d --- /dev/null +++ b/shims/spark34/src/main/resources/META-INF/services/io.glutenproject.sql.shims.SparkShimProvider @@ -0,0 +1 @@ +io.glutenproject.sql.shims.spark34.SparkShimProvider \ No newline at end of file diff --git a/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala new file mode 100644 index 000000000000..cdc42f3b43fd --- /dev/null +++ b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/Spark34Shims.scala @@ -0,0 +1,139 @@ +/* + * 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 io.glutenproject.sql.shims.spark34 + +import io.glutenproject.GlutenConfig +import io.glutenproject.expression.{ExpressionNames, Sig} +import io.glutenproject.expression.ExpressionNames.EMPTY2NULL +import io.glutenproject.sql.shims.{ShimDescriptor, SparkShims} + +import org.apache.spark.SparkException +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Distribution} +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.execution.{FileSourceScanLike, PartitionedFileUtil, SparkPlan} +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.{BucketingUtils, FilePartition, FileScanRDD, PartitionDirectory, PartitionedFile, PartitioningAwareFileIndex} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.text.TextScan +import org.apache.spark.sql.execution.datasources.v2.utils.CatalogUtil +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import org.apache.hadoop.fs.Path + +class Spark34Shims extends SparkShims { + override def getShimDescriptor: ShimDescriptor = SparkShimProvider.DESCRIPTOR + + override def getDistribution( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression]): Seq[Distribution] = { + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + } + + override def expressionMappings: Seq[Sig] = { + val list = if (GlutenConfig.getConf.enableNativeBloomFilter) { + Seq( + Sig[BloomFilterMightContain](ExpressionNames.MIGHT_CONTAIN), + Sig[BloomFilterAggregate](ExpressionNames.BLOOM_FILTER_AGG)) + } else Seq.empty + list ++ Seq( + Sig[SplitPart](ExpressionNames.SPLIT_PART), + Sig[Sec](ExpressionNames.SEC), + Sig[Csc](ExpressionNames.CSC), + Sig[Empty2Null](ExpressionNames.EMPTY2NULL)) + + } + + override def convertPartitionTransforms( + partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + CatalogUtil.convertPartitionTransforms(partitions) + } + + override def generateFileScanRDD( + sparkSession: SparkSession, + readFunction: PartitionedFile => Iterator[InternalRow], + filePartitions: Seq[FilePartition], + fileSourceScanExec: FileSourceScanExec): FileScanRDD = { + new FileScanRDD( + sparkSession, + readFunction, + filePartitions, + new StructType( + fileSourceScanExec.requiredSchema.fields ++ + fileSourceScanExec.relation.partitionSchema.fields), + fileSourceScanExec.fileConstantMetadataColumns + ) + } + + override def getTextScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): TextScan = { + new TextScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + options, + partitionFilters, + dataFilters) + } + + override def filesGroupedToBuckets( + selectedPartitions: Array[PartitionDirectory]): Map[Int, Array[PartitionedFile]] = { + selectedPartitions + .flatMap { + p => p.files.map(f => PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values)) + } + .groupBy { + f => + BucketingUtils + .getBucketId(f.toPath.getName) + .getOrElse(throw invalidBucketFile(f.toPath.getName)) + } + } + + override def getBatchScanExecTable(batchScan: BatchScanExec): Table = batchScan.table + + override def generatePartitionedFile( + partitionValues: InternalRow, + filePath: String, + start: Long, + length: Long, + @transient locations: Array[String] = Array.empty): PartitionedFile = + PartitionedFile(partitionValues, SparkPath.fromPathString(filePath), start, length, locations) + + private def invalidBucketFile(path: String): Throwable = { + new SparkException( + errorClass = "INVALID_BUCKET_FILE", + messageParameters = Map("error" -> path), + cause = null) + } +} diff --git a/gluten-core/src/main/scala/io/glutenproject/utils/ColumnarShuffleUtil.scala b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/SparkShimProvider.scala similarity index 50% rename from gluten-core/src/main/scala/io/glutenproject/utils/ColumnarShuffleUtil.scala rename to shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/SparkShimProvider.scala index 8e6d47ec80ef..1aaa46f4a8e1 100644 --- a/gluten-core/src/main/scala/io/glutenproject/utils/ColumnarShuffleUtil.scala +++ b/shims/spark34/src/main/scala/io/glutenproject/sql/shims/spark34/SparkShimProvider.scala @@ -14,32 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package io.glutenproject.utils +package io.glutenproject.sql.shims.spark34 -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.{ColumnarShuffleExchangeExec, SparkPlan} -import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import io.glutenproject.sql.shims.{SparkShimDescriptor, SparkShims} +import io.glutenproject.sql.shims.spark34.SparkShimProvider.DESCRIPTOR -object ColumnarShuffleUtil { +object SparkShimProvider { + val DESCRIPTOR = SparkShimDescriptor(3, 4, 1) +} + +class SparkShimProvider extends io.glutenproject.sql.shims.SparkShimProvider { + def createShim: SparkShims = { + new Spark34Shims() + } - /** - * Generate a columnar plan for shuffle exchange. - * - * @param plan - * the spark plan of shuffle exchange. - * @param child - * the child of shuffle exchange. - * @return - * a columnar shuffle exchange. - */ - def genColumnarShuffleExchange( - plan: ShuffleExchangeExec, - child: SparkPlan, - shuffleOutputAttributes: Seq[Attribute]): SparkPlan = { - ColumnarShuffleExchangeExec( - plan.outputPartitioning, - child, - plan.shuffleOrigin, - shuffleOutputAttributes) + def matches(version: String): Boolean = { + val majorMinorVersionMatch = DESCRIPTOR.toMajorMinorVersion == + extractMajorAndMinorVersion(version) + if (majorMinorVersionMatch && DESCRIPTOR.toString() != version) { + logWarning( + s"Spark runtime version $version is not matched with Gluten's fully" + + s" tested version ${DESCRIPTOR.toString()}") + } + majorMinorVersionMatch } } diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/catalyst/expressions/PromotePrecision.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/catalyst/expressions/PromotePrecision.scala new file mode 100644 index 000000000000..8de5a07fe045 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/catalyst/expressions/PromotePrecision.scala @@ -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. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +case class PromotePrecision(child: Expression) extends UnaryExpression { + override def dataType: DataType = child.dataType + override def eval(input: InternalRow): Any = child.eval(input) + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + child.genCode(ctx) + override def prettyName: String = "promote_precision" + override def sql: String = child.sql + override lazy val canonicalized: Expression = child.canonicalized + + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala new file mode 100644 index 000000000000..0a62c41a69df --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/FileSourceScanExecShim.scala @@ -0,0 +1,127 @@ +/* + * 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 io.glutenproject.metrics.GlutenTimeMetric + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, BoundReference, DynamicPruningExpression, Expression, PlanExpression, Predicate} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} +import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.collection.BitSet + +class FileSourceScanExecShim( + @transient relation: HadoopFsRelation, + output: Seq[Attribute], + requiredSchema: StructType, + partitionFilters: Seq[Expression], + optionalBucketSet: Option[BitSet], + optionalNumCoalescedBuckets: Option[Int], + dataFilters: Seq[Expression], + tableIdentifier: Option[TableIdentifier], + disableBucketedScan: Boolean = false) + extends FileSourceScanExec( + relation, + output, + requiredSchema, + partitionFilters, + optionalBucketSet, + optionalNumCoalescedBuckets, + dataFilters, + tableIdentifier, + disableBucketedScan) { + + // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. + @transient override lazy val metrics: Map[String, SQLMetric] = Map() + + override def equals(other: Any): Boolean = other match { + case that: FileSourceScanExecShim => + (that.canEqual(this)) && super.equals(that) + case _ => false + } + + override def hashCode(): Int = super.hashCode() + + override def canEqual(other: Any): Boolean = other.isInstanceOf[FileSourceScanExecShim] + + def hasMetadataColumns: Boolean = fileConstantMetadataColumns.nonEmpty + + def hasFieldIds: Boolean = ParquetUtils.hasFieldIds(requiredSchema) + + private def isDynamicPruningFilter(e: Expression): Boolean = + e.find(_.isInstanceOf[PlanExpression[_]]).isDefined + + protected def setFilesNumAndSizeMetric( + partitions: Seq[PartitionDirectory], + static: Boolean): Unit = { + val filesNum = partitions.map(_.files.size.toLong).sum + val filesSize = partitions.map(_.files.map(_.getLen).sum).sum + if (!static || !partitionFilters.exists(isDynamicPruningFilter)) { + driverMetrics("numFiles").set(filesNum) + driverMetrics("filesSize").set(filesSize) + } else { + driverMetrics("staticFilesNum").set(filesNum) + driverMetrics("staticFilesSize").set(filesSize) + } + if (relation.partitionSchema.nonEmpty) { + driverMetrics("numPartitions").set(partitions.length) + } + } + + @transient override lazy val dynamicallySelectedPartitions: Array[PartitionDirectory] = { + val dynamicPartitionFilters = + partitionFilters.filter(isDynamicPruningFilter) + val selected = if (dynamicPartitionFilters.nonEmpty) { + // When it includes some DynamicPruningExpression, + // it needs to execute InSubqueryExec first, + // because doTransform path can't execute 'doExecuteColumnar' which will + // execute prepare subquery first. + dynamicPartitionFilters.foreach { + case DynamicPruningExpression(inSubquery: InSubqueryExec) => + if (inSubquery.values().isEmpty) inSubquery.updateResult() + case e: Expression => + e.foreach { + case s: ScalarSubquery => s.updateResult() + case _ => + } + case _ => + } + GlutenTimeMetric.withMillisTime { + // call the file index for the files matching all filters except dynamic partition filters + val predicate = dynamicPartitionFilters.reduce(And) + val partitionColumns = relation.partitionSchema + val boundPredicate = Predicate.create( + predicate.transform { + case a: AttributeReference => + val index = partitionColumns.indexWhere(a.name == _.name) + BoundReference(index, partitionColumns(index).dataType, nullable = true) + }, + Nil + ) + val ret = selectedPartitions.filter(p => boundPredicate.eval(p.values)) + setFilesNumAndSizeMetric(ret, static = false) + ret + }(t => driverMetrics("pruningTime").set(t)) + } else { + selectedPartitions + } + sendDriverMetrics() + selected + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/JoinSelectionShim.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/JoinSelectionShim.scala new file mode 100644 index 000000000000..20b9dea333a5 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/JoinSelectionShim.scala @@ -0,0 +1,51 @@ +/* + * 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.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint, LogicalPlan} + +// https://issues.apache.org/jira/browse/SPARK-36745 +object JoinSelectionShim { + object ExtractEquiJoinKeysShim { + type ReturnType = + ( + JoinType, + Seq[Expression], + Seq[Expression], + Option[Expression], + LogicalPlan, + LogicalPlan, + JoinHint) + def unapply(join: Join): Option[ReturnType] = { + ExtractEquiJoinKeys.unapply(join).map { + case ( + joinType, + leftKeys, + rightKeys, + otherPredicates, + predicatesOfJoinKeys, + left, + right, + hint) => + (joinType, leftKeys, rightKeys, otherPredicates, left, right, hint) + } + } + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala new file mode 100644 index 000000000000..b893e89270d5 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.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.execution.datasources + +import org.apache.spark.{SparkContext, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker._ +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.util.SerializableConfiguration + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} + +import java.io.FileNotFoundException +import java.nio.charset.StandardCharsets + +import scala.collection.mutable + +/** + * Simple metrics collected during an instance of [[FileFormatDataWriter]]. These were first + * introduced in https://github.com/apache/spark/pull/18159 (SPARK-20703). + */ +case class BasicWriteTaskStats( + partitions: Seq[InternalRow], + numFiles: Int, + numBytes: Long, + numRows: Long) + extends WriteTaskStats + +/** Simple [[WriteTaskStatsTracker]] implementation that produces [[BasicWriteTaskStats]]. */ +class BasicWriteTaskStatsTracker( + hadoopConf: Configuration, + taskCommitTimeMetric: Option[SQLMetric] = None) + extends WriteTaskStatsTracker + with Logging { + + private[this] val partitions: mutable.ArrayBuffer[InternalRow] = mutable.ArrayBuffer.empty + private[this] var numFiles: Int = 0 + private[this] var numSubmittedFiles: Int = 0 + private[this] var numBytes: Long = 0L + private[this] var numRows: Long = 0L + + private[this] val submittedFiles = mutable.HashSet[String]() + + /** + * Get the size of the file expected to have been written by a worker. + * @param filePath + * path to the file + * @return + * the file size or None if the file was not found. + */ + private def getFileSize(filePath: String): Option[Long] = { + val path = new Path(filePath) + val fs = path.getFileSystem(hadoopConf) + getFileSize(fs, path) + } + + /** + * Get the size of the file expected to have been written by a worker. This supports the XAttr in + * HADOOP-17414 when the "magic committer" adds a custom HTTP header to the a zero byte marker. If + * the output file as returned by getFileStatus > 0 then the length if returned. For zero-byte + * files, the (optional) Hadoop FS API getXAttr() is invoked. If a parseable, non-negative length + * can be retrieved, this is returned instead of the length. + * @return + * the file size or None if the file was not found. + */ + private[datasources] def getFileSize(fs: FileSystem, path: Path): Option[Long] = { + // the normal file status probe. + try { + val len = fs.getFileStatus(path).getLen + if (len > 0) { + return Some(len) + } + } catch { + case e: FileNotFoundException => + // may arise against eventually consistent object stores. + logDebug(s"File $path is not yet visible", e) + return None + } + + // Output File Size is 0. Look to see if it has an attribute + // declaring a future-file-length. + // Failure of API call, parsing, invalid value all return the + // 0 byte length. + + var len = 0L + try { + val attr = fs.getXAttr(path, BasicWriteJobStatsTracker.FILE_LENGTH_XATTR) + if (attr != null && attr.nonEmpty) { + val str = new String(attr, StandardCharsets.UTF_8) + logDebug(s"File Length statistics for $path retrieved from XAttr: $str") + // a non-empty header was found. parse to a long via the java class + val l = java.lang.Long.parseLong(str) + if (l > 0) { + len = l + } else { + logDebug("Ignoring negative value in XAttr file length") + } + } + } catch { + case e: NumberFormatException => + // warn but don't dump the whole stack + logInfo( + s"Failed to parse" + + s" ${BasicWriteJobStatsTracker.FILE_LENGTH_XATTR}:$e;" + + s" bytes written may be under-reported"); + case e: UnsupportedOperationException => + // this is not unusual; ignore + logDebug(s"XAttr not supported on path $path", e); + case e: Exception => + // Something else. Log at debug and continue. + logDebug(s"XAttr processing failure on $path", e); + } + Some(len) + } + + override def newPartition(partitionValues: InternalRow): Unit = { + partitions.append(partitionValues) + } + + override def newFile(filePath: String): Unit = { + submittedFiles += filePath + numSubmittedFiles += 1 + } + + override def closeFile(filePath: String): Unit = { + updateFileStats(filePath) + submittedFiles.remove(filePath) + } + + private def updateFileStats(filePath: String): Unit = { + getFileSize(filePath).foreach { + len => + numBytes += len + numFiles += 1 + } + } + + override def newRow(filePath: String, row: InternalRow): Unit = row match { + case fake: FakeRow => + numRows += fake.batch.numRows() + case _ => numRows += 1 + } + + override def getFinalStats(taskCommitTime: Long): WriteTaskStats = { + submittedFiles.foreach(updateFileStats) + submittedFiles.clear() + + // Reports bytesWritten and recordsWritten to the Spark output metrics. + Option(TaskContext.get()).map(_.taskMetrics().outputMetrics).foreach { + outputMetrics => + outputMetrics.setBytesWritten(numBytes) + outputMetrics.setRecordsWritten(numRows) + } + + if (numSubmittedFiles != numFiles) { + logWarning( + s"Expected $numSubmittedFiles files, but only saw $numFiles. " + + "This could be due to the output format not writing empty files, " + + "or files being not immediately visible in the filesystem.") + } + taskCommitTimeMetric.foreach(_ += taskCommitTime) + BasicWriteTaskStats(partitions.toSeq, numFiles, numBytes, numRows) + } +} + +/** + * Simple [[WriteJobStatsTracker]] implementation that's serializable, capable of instantiating + * [[BasicWriteTaskStatsTracker]] on executors and processing the [[BasicWriteTaskStats]] they + * produce by aggregating the metrics and posting them as DriverMetricUpdates. + */ +class BasicWriteJobStatsTracker( + serializableHadoopConf: SerializableConfiguration, + @transient val driverSideMetrics: Map[String, SQLMetric], + taskCommitTimeMetric: SQLMetric) + extends WriteJobStatsTracker { + + def this(serializableHadoopConf: SerializableConfiguration, metrics: Map[String, SQLMetric]) = { + this(serializableHadoopConf, metrics - TASK_COMMIT_TIME, metrics(TASK_COMMIT_TIME)) + } + + override def newTaskInstance(): WriteTaskStatsTracker = { + new BasicWriteTaskStatsTracker(serializableHadoopConf.value, Some(taskCommitTimeMetric)) + } + + override def processStats(stats: Seq[WriteTaskStats], jobCommitTime: Long): Unit = { + val sparkContext = SparkContext.getActive.get + val partitionsSet: mutable.Set[InternalRow] = mutable.HashSet.empty + var numFiles: Long = 0L + var totalNumBytes: Long = 0L + var totalNumOutput: Long = 0L + + val basicStats = stats.map(_.asInstanceOf[BasicWriteTaskStats]) + + basicStats.foreach { + summary => + partitionsSet ++= summary.partitions + numFiles += summary.numFiles + totalNumBytes += summary.numBytes + totalNumOutput += summary.numRows + } + + driverSideMetrics(JOB_COMMIT_TIME).add(jobCommitTime) + driverSideMetrics(NUM_FILES_KEY).add(numFiles) + driverSideMetrics(NUM_OUTPUT_BYTES_KEY).add(totalNumBytes) + driverSideMetrics(NUM_OUTPUT_ROWS_KEY).add(totalNumOutput) + driverSideMetrics(NUM_PARTS_KEY).add(partitionsSet.size) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, driverSideMetrics.values.toList) + } +} + +object BasicWriteJobStatsTracker { + private val NUM_FILES_KEY = "numFiles" + private val NUM_OUTPUT_BYTES_KEY = "numOutputBytes" + private val NUM_OUTPUT_ROWS_KEY = "numOutputRows" + private val NUM_PARTS_KEY = "numParts" + val TASK_COMMIT_TIME = "taskCommitTime" + val JOB_COMMIT_TIME = "jobCommitTime" + + /** XAttr key of the data length header added in HADOOP-17414. */ + val FILE_LENGTH_XATTR = "header.x-hadoop-s3a-magic-data-length" + + def metrics: Map[String, SQLMetric] = { + val sparkContext = SparkContext.getActive.get + Map( + NUM_FILES_KEY -> SQLMetrics.createMetric(sparkContext, "number of written files"), + NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createSizeMetric(sparkContext, "written output"), + NUM_OUTPUT_ROWS_KEY -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + NUM_PARTS_KEY -> SQLMetrics.createMetric(sparkContext, "number of dynamic part"), + TASK_COMMIT_TIME -> SQLMetrics.createTimingMetric(sparkContext, "task commit time"), + JOB_COMMIT_TIME -> SQLMetrics.createTimingMetric(sparkContext, "job commit time") + ) + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala new file mode 100644 index 000000000000..2d54cffd7ef9 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -0,0 +1,675 @@ +/* + * 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.datasources + +import io.glutenproject.execution.datasource.GlutenRowSplitter + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.{FileCommitProtocol, FileNameSpec} +import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.connector.write.{DataWriter, WriterCommitMessage} +import org.apache.spark.sql.execution.datasources.FileFormatWriter.ConcurrentOutputWriterSpec +import org.apache.spark.sql.execution.metric.{CustomMetrics, SQLMetric} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StringType +import org.apache.spark.util.{SerializableConfiguration, Utils} + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import scala.collection.mutable + +/** + * Abstract class for writing out data in a single Spark task. Exceptions thrown by the + * implementation of this trait will automatically trigger task aborts. + */ +abstract class FileFormatDataWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + customMetrics: Map[String, SQLMetric]) + extends DataWriter[InternalRow] { + + /** + * Max number of files a single task writes out due to file size. In most cases the number of + * files written should be very small. This is just a safe guard to protect some really bad + * settings, e.g. maxRecordsPerFile = 1. + */ + protected val MAX_FILE_COUNTER: Int = 1000 * 1000 + protected val updatedPartitions: mutable.Set[String] = mutable.Set[String]() + protected var currentWriter: OutputWriter = _ + + /** Trackers for computing various statistics on the data as it's being written out. */ + protected val statsTrackers: Seq[WriteTaskStatsTracker] = + description.statsTrackers.map(_.newTaskInstance()) + + /** Release resources of `currentWriter`. */ + protected def releaseCurrentWriter(): Unit = { + if (currentWriter != null) { + try { + currentWriter.close() + statsTrackers.foreach(_.closeFile(currentWriter.path())) + } finally { + currentWriter = null + } + } + } + + /** Release all resources. */ + protected def releaseResources(): Unit = { + // Call `releaseCurrentWriter()` by default, as this is the only resource to be released. + releaseCurrentWriter() + } + + /** Writes a record. */ + def write(record: InternalRow): Unit + + def writeWithMetrics(record: InternalRow, count: Long): Unit = { + if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) { + CustomMetrics.updateMetrics(currentMetricsValues, customMetrics) + } + write(record) + } + + /** Write an iterator of records. */ + def writeWithIterator(iterator: Iterator[InternalRow]): Unit = { + var count = 0L + while (iterator.hasNext) { + writeWithMetrics(iterator.next(), count) + count += 1 + } + CustomMetrics.updateMetrics(currentMetricsValues, customMetrics) + } + + /** + * Returns the summary of relative information which includes the list of partition strings + * written out. The list of partitions is sent back to the driver and used to update the catalog. + * Other information will be sent back to the driver too and used to e.g. update the metrics in + * UI. + */ + override def commit(): WriteTaskResult = { + releaseResources() + val (taskCommitMessage, taskCommitTime) = Utils.timeTakenMs { + committer.commitTask(taskAttemptContext) + } + val summary = ExecutedWriteSummary( + updatedPartitions = updatedPartitions.toSet, + stats = statsTrackers.map(_.getFinalStats(taskCommitTime))) + WriteTaskResult(taskCommitMessage, summary) + } + + def abort(): Unit = { + try { + releaseResources() + } finally { + committer.abortTask(taskAttemptContext) + } + } + + override def close(): Unit = {} +} + +/** FileFormatWriteTask for empty partitions */ +class EmptyDirectoryDataWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + customMetrics: Map[String, SQLMetric] = Map.empty +) extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { + override def write(record: InternalRow): Unit = {} +} + +/** Writes data to a single directory (used for non-dynamic-partition writes). */ +class SingleDirectoryDataWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + customMetrics: Map[String, SQLMetric] = Map.empty) + extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { + private var fileCounter: Int = _ + private var recordsInFile: Long = _ + // Initialize currentWriter and statsTrackers + newOutputWriter() + + private def newOutputWriter(): Unit = { + recordsInFile = 0 + releaseResources() + + val ext = description.outputWriterFactory.getFileExtension(taskAttemptContext) + val currentPath = + committer.newTaskTempFile(taskAttemptContext, None, f"-c$fileCounter%03d" + ext) + + currentWriter = description.outputWriterFactory.newInstance( + path = currentPath, + dataSchema = description.dataColumns.toStructType, + context = taskAttemptContext) + + statsTrackers.foreach(_.newFile(currentPath)) + } + + private def updateRecordsInFile(record: InternalRow): Unit = record match { + case fake: FakeRow => + recordsInFile += fake.batch.numRows() + case _ => recordsInFile += 1 + } + + override def write(record: InternalRow): Unit = { + if (description.maxRecordsPerFile > 0 && recordsInFile >= description.maxRecordsPerFile) { + fileCounter += 1 + assert( + fileCounter < MAX_FILE_COUNTER, + s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") + + newOutputWriter() + } + + currentWriter.write(record) + statsTrackers.foreach(_.newRow(currentWriter.path, record)) + updateRecordsInFile(record) + } +} + +/** + * Holds common logic for writing data with dynamic partition writes, meaning it can write to + * multiple directories (partitions) or files (bucketing). + */ +abstract class BaseDynamicPartitionDataWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + customMetrics: Map[String, SQLMetric]) + extends FileFormatDataWriter(description, taskAttemptContext, committer, customMetrics) { + + /** Flag saying whether or not the data to be written out is partitioned. */ + protected val isPartitioned = description.partitionColumns.nonEmpty + + /** Flag saying whether or not the data to be written out is bucketed. */ + protected val isBucketed = description.bucketSpec.isDefined + + assert( + isPartitioned || isBucketed, + s"""DynamicPartitionWriteTask should be used for writing out data that's either + |partitioned or bucketed. In this case neither is true. + |WriteJobDescription: $description + """.stripMargin + ) + + /** Number of records in current file. */ + protected var recordsInFile: Long = _ + + /** + * File counter for writing current partition or bucket. For same partition or bucket, we may have + * more than one file, due to number of records limit per file. + */ + protected var fileCounter: Int = _ + + /** Extracts the partition values out of an input row. */ + protected lazy val getPartitionValues: InternalRow => UnsafeRow = { + val proj = UnsafeProjection.create(description.partitionColumns, description.allColumns) + row => proj(row) + } + + /** Expression that given partition columns builds a path string like: col1=val/col2=val/... */ + private lazy val partitionPathExpression: Expression = Concat( + description.partitionColumns.zipWithIndex.flatMap { + case (c, i) => + val partitionName = ScalaUDF( + ExternalCatalogUtils.getPartitionPathString _, + StringType, + Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) + if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) + }) + + /** + * Evaluates the `partitionPathExpression` above on a row of `partitionValues` and returns the + * partition string. + */ + private lazy val getPartitionPath: InternalRow => String = { + val proj = UnsafeProjection.create(Seq(partitionPathExpression), description.partitionColumns) + row => proj(row).getString(0) + } + + /** Given an input row, returns the corresponding `bucketId` */ + protected lazy val getBucketId: InternalRow => Int = { + val proj = + UnsafeProjection.create( + Seq(description.bucketSpec.get.bucketIdExpression), + description.allColumns) + row => proj(row).getInt(0) + } + + /** Returns the data columns to be written given an input row */ + protected val getOutputRow = + UnsafeProjection.create(description.dataColumns, description.allColumns) + + /** + * Opens a new OutputWriter given a partition key and/or a bucket id. If bucket id is specified, + * we will append it to the end of the file name, but before the file extension, e.g. + * part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet + * + * @param partitionValues + * the partition which all tuples being written by this OutputWriter belong to + * @param bucketId + * the bucket which all tuples being written by this OutputWriter belong to + * @param closeCurrentWriter + * close and release resource for current writer + */ + protected def renewCurrentWriter( + partitionValues: Option[InternalRow], + bucketId: Option[Int], + closeCurrentWriter: Boolean): Unit = { + + recordsInFile = 0 + if (closeCurrentWriter) { + releaseCurrentWriter() + } + + val partDir = partitionValues.map(getPartitionPath(_)) + partDir.foreach(updatedPartitions.add) + + val bucketIdStr = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + + // The prefix and suffix must be in a form that matches our bucketing format. See BucketingUtils + // for details. The prefix is required to represent bucket id when writing Hive-compatible + // bucketed table. + val prefix = bucketId match { + case Some(id) => description.bucketSpec.get.bucketFileNamePrefix(id) + case _ => "" + } + val suffix = f"$bucketIdStr.c$fileCounter%03d" + + description.outputWriterFactory.getFileExtension(taskAttemptContext) + val fileNameSpec = FileNameSpec(prefix, suffix) + + val customPath = partDir.flatMap { + dir => description.customPartitionLocations.get(PartitioningUtils.parsePathFragment(dir)) + } + val currentPath = if (customPath.isDefined) { + committer.newTaskTempFileAbsPath(taskAttemptContext, customPath.get, fileNameSpec) + } else { + committer.newTaskTempFile(taskAttemptContext, partDir, fileNameSpec) + } + + currentWriter = description.outputWriterFactory.newInstance( + path = currentPath, + dataSchema = description.dataColumns.toStructType, + context = taskAttemptContext) + + statsTrackers.foreach(_.newFile(currentPath)) + } + + /** + * Open a new output writer when number of records exceeding limit. + * + * @param partitionValues + * the partition which all tuples being written by this `OutputWriter` belong to + * @param bucketId + * the bucket which all tuples being written by this `OutputWriter` belong to + */ + protected def renewCurrentWriterIfTooManyRecords( + partitionValues: Option[InternalRow], + bucketId: Option[Int]): Unit = { + // Exceeded the threshold in terms of the number of records per file. + // Create a new file by increasing the file counter. + fileCounter += 1 + assert( + fileCounter < MAX_FILE_COUNTER, + s"File counter $fileCounter is beyond max value $MAX_FILE_COUNTER") + renewCurrentWriter(partitionValues, bucketId, closeCurrentWriter = true) + } + + protected def updateRecordsInFile(record: InternalRow): Unit = record match { + case fake: FakeRow => + recordsInFile += fake.batch.numRows() + case _ => recordsInFile += 1 + } + + /** + * Writes the given record with current writer. + * + * @param record + * The record to write + */ + protected def writeRecord(record: InternalRow): Unit = { + val outputRow = getOutputRow(record) + currentWriter.write(outputRow) + statsTrackers.foreach(_.newRow(currentWriter.path, outputRow)) + recordsInFile += 1 + } +} + +/** + * Dynamic partition writer with single writer, meaning only one writer is opened at any time for + * writing. The records to be written are required to be sorted on partition and/or bucket column(s) + * before writing. + */ +class DynamicPartitionDataSingleWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + customMetrics: Map[String, SQLMetric] = Map.empty) + extends BaseDynamicPartitionDataWriter( + description, + taskAttemptContext, + committer, + customMetrics) { + + private var currentPartitionValues: Option[UnsafeRow] = None + private var currentBucketId: Option[Int] = None + + private val partitionColIndice: Array[Int] = + description.partitionColumns.flatMap { + pcol => + description.allColumns.zipWithIndex.collect { + case (acol, index) if acol.name == pcol.name && acol.exprId == pcol.exprId => index + } + }.toArray + + private def beforeWrite(record: InternalRow): Unit = { + val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None + val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None + + if (currentPartitionValues != nextPartitionValues || currentBucketId != nextBucketId) { + // See a new partition or bucket - write to a new partition dir (or a new bucket file). + if (isPartitioned && currentPartitionValues != nextPartitionValues) { + currentPartitionValues = Some(nextPartitionValues.get.copy()) + statsTrackers.foreach(_.newPartition(currentPartitionValues.get)) + } + if (isBucketed) { + currentBucketId = nextBucketId + } + + fileCounter = 0 + renewCurrentWriter(currentPartitionValues, currentBucketId, closeCurrentWriter = true) + } else if ( + description.maxRecordsPerFile > 0 && + recordsInFile >= description.maxRecordsPerFile + ) { + renewCurrentWriterIfTooManyRecords(currentPartitionValues, currentBucketId) + } + } + + override def write(record: InternalRow): Unit = { + record match { + case fakeRow: FakeRow => + if (fakeRow.batch.numRows() > 0) { + val blockStripes = GlutenRowSplitter.getInstance + .splitBlockByPartitionAndBucket(fakeRow, partitionColIndice, isBucketed) + + val iter = blockStripes.iterator() + while (iter.hasNext) { + val blockStripe = iter.next() + val headingRow = blockStripe.getHeadingRow + beforeWrite(headingRow) + writeStripe(new FakeRow(blockStripe.getColumnarBatch)) + } + blockStripes.release() + } + case _ => + beforeWrite(record) + writeRecord(record) + } + } + + protected def writeStripe(record: InternalRow): Unit = { + currentWriter.write(record) + statsTrackers.foreach(_.newRow(currentWriter.path, record)) + updateRecordsInFile(record) + } +} + +/** + * Dynamic partition writer with concurrent writers, meaning multiple concurrent writers are opened + * for writing. + * + * The process has the following steps: + * - Step 1: Maintain a map of output writers per each partition and/or bucket columns. Keep all + * writers opened and write rows one by one. + * - Step 2: If number of concurrent writers exceeds limit, sort rest of rows on partition and/or + * bucket column(s). Write rows one by one, and eagerly close the writer when finishing each + * partition and/or bucket. + * + * Caller is expected to call `writeWithIterator()` instead of `write()` to write records. + */ +class DynamicPartitionDataConcurrentWriter( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + committer: FileCommitProtocol, + concurrentOutputWriterSpec: ConcurrentOutputWriterSpec, + customMetrics: Map[String, SQLMetric] = Map.empty) + extends BaseDynamicPartitionDataWriter(description, taskAttemptContext, committer, customMetrics) + with Logging { + + /** Wrapper class to index a unique concurrent output writer. */ + private case class WriterIndex(var partitionValues: Option[UnsafeRow], var bucketId: Option[Int]) + + /** Wrapper class for status of a unique concurrent output writer. */ + private class WriterStatus( + var outputWriter: OutputWriter, + var recordsInFile: Long, + var fileCounter: Int) + + /** + * State to indicate if we are falling back to sort-based writer. Because we first try to use + * concurrent writers, its initial value is false. + */ + private var sorted: Boolean = false + private val concurrentWriters = mutable.HashMap[WriterIndex, WriterStatus]() + + /** + * The index for current writer. Intentionally make the index mutable and reusable. Avoid JVM GC + * issue when many short-living `WriterIndex` objects are created if switching between concurrent + * writers frequently. + */ + private val currentWriterId = WriterIndex(None, None) + + /** Release resources for all concurrent output writers. */ + override protected def releaseResources(): Unit = { + currentWriter = null + concurrentWriters.values.foreach( + status => { + if (status.outputWriter != null) { + try { + status.outputWriter.close() + } finally { + status.outputWriter = null + } + } + }) + concurrentWriters.clear() + } + + override def write(record: InternalRow): Unit = { + val nextPartitionValues = if (isPartitioned) Some(getPartitionValues(record)) else None + val nextBucketId = if (isBucketed) Some(getBucketId(record)) else None + + if ( + currentWriterId.partitionValues != nextPartitionValues || + currentWriterId.bucketId != nextBucketId + ) { + // See a new partition or bucket - write to a new partition dir (or a new bucket file). + if (currentWriter != null) { + if (!sorted) { + // Update writer status in concurrent writers map, because the writer is probably needed + // again later for writing other rows. + updateCurrentWriterStatusInMap() + } else { + // Remove writer status in concurrent writers map and release current writer resource, + // because the writer is not needed any more. + concurrentWriters.remove(currentWriterId) + releaseCurrentWriter() + } + } + + if (isBucketed) { + currentWriterId.bucketId = nextBucketId + } + if (isPartitioned && currentWriterId.partitionValues != nextPartitionValues) { + currentWriterId.partitionValues = Some(nextPartitionValues.get.copy()) + if (!concurrentWriters.contains(currentWriterId)) { + statsTrackers.foreach(_.newPartition(currentWriterId.partitionValues.get)) + } + } + setupCurrentWriterUsingMap() + } + + if ( + description.maxRecordsPerFile > 0 && + recordsInFile >= description.maxRecordsPerFile + ) { + renewCurrentWriterIfTooManyRecords(currentWriterId.partitionValues, currentWriterId.bucketId) + // Update writer status in concurrent writers map, as a new writer is created. + updateCurrentWriterStatusInMap() + } + writeRecord(record) + } + + /** Write iterator of records with concurrent writers. */ + override def writeWithIterator(iterator: Iterator[InternalRow]): Unit = { + var count = 0L + while (iterator.hasNext && !sorted) { + writeWithMetrics(iterator.next(), count) + count += 1 + } + CustomMetrics.updateMetrics(currentMetricsValues, customMetrics) + + if (iterator.hasNext) { + count = 0L + clearCurrentWriterStatus() + val sorter = concurrentOutputWriterSpec.createSorter() + val sortIterator = sorter.sort(iterator.asInstanceOf[Iterator[UnsafeRow]]) + while (sortIterator.hasNext) { + writeWithMetrics(sortIterator.next(), count) + count += 1 + } + CustomMetrics.updateMetrics(currentMetricsValues, customMetrics) + } + } + + /** Update current writer status in map. */ + private def updateCurrentWriterStatusInMap(): Unit = { + val status = concurrentWriters(currentWriterId) + status.outputWriter = currentWriter + status.recordsInFile = recordsInFile + status.fileCounter = fileCounter + } + + /** Retrieve writer in map, or create a new writer if not exists. */ + private def setupCurrentWriterUsingMap(): Unit = { + if (concurrentWriters.contains(currentWriterId)) { + val status = concurrentWriters(currentWriterId) + currentWriter = status.outputWriter + recordsInFile = status.recordsInFile + fileCounter = status.fileCounter + } else { + fileCounter = 0 + renewCurrentWriter( + currentWriterId.partitionValues, + currentWriterId.bucketId, + closeCurrentWriter = false) + if (!sorted) { + assert( + concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters, + s"Number of concurrent output file writers is ${concurrentWriters.size} " + + s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters}" + ) + } else { + assert( + concurrentWriters.size <= concurrentOutputWriterSpec.maxWriters + 1, + s"Number of output file writers after sort is ${concurrentWriters.size} " + + s" which is beyond max value ${concurrentOutputWriterSpec.maxWriters + 1}" + ) + } + concurrentWriters.put( + currentWriterId.copy(), + new WriterStatus(currentWriter, recordsInFile, fileCounter)) + if (concurrentWriters.size >= concurrentOutputWriterSpec.maxWriters && !sorted) { + // Fall back to sort-based sequential writer mode. + logInfo( + s"Number of concurrent writers ${concurrentWriters.size} reaches the threshold. " + + "Fall back from concurrent writers to sort-based sequential writer. You may change " + + s"threshold with configuration ${SQLConf.MAX_CONCURRENT_OUTPUT_FILE_WRITERS.key}") + sorted = true + } + } + } + + /** Clear the current writer status in map. */ + private def clearCurrentWriterStatus(): Unit = { + if (currentWriterId.partitionValues.isDefined || currentWriterId.bucketId.isDefined) { + updateCurrentWriterStatusInMap() + } + currentWriterId.partitionValues = None + currentWriterId.bucketId = None + currentWriter = null + recordsInFile = 0 + fileCounter = 0 + } +} + +/** + * Bucketing specification for all the write tasks. + * + * @param bucketIdExpression + * Expression to calculate bucket id based on bucket column(s). + * @param bucketFileNamePrefix + * Prefix of output file name based on bucket id. + */ +case class WriterBucketSpec(bucketIdExpression: Expression, bucketFileNamePrefix: Int => String) + +/** A shared job description for all the write tasks. */ +class WriteJobDescription( + val uuid: String, // prevent collision between different (appending) write jobs + val serializableHadoopConf: SerializableConfiguration, + val outputWriterFactory: OutputWriterFactory, + val allColumns: Seq[Attribute], + val dataColumns: Seq[Attribute], + val partitionColumns: Seq[Attribute], + val bucketSpec: Option[WriterBucketSpec], + val path: String, + val customPartitionLocations: Map[TablePartitionSpec, String], + val maxRecordsPerFile: Long, + val timeZoneId: String, + val statsTrackers: Seq[WriteJobStatsTracker]) + extends Serializable { + + assert( + AttributeSet(allColumns) == AttributeSet(partitionColumns ++ dataColumns), + s""" + |All columns: ${allColumns.mkString(", ")} + |Partition columns: ${partitionColumns.mkString(", ")} + |Data columns: ${dataColumns.mkString(", ")} + """.stripMargin + ) +} + +/** The result of a successful write task. */ +case class WriteTaskResult(commitMsg: TaskCommitMessage, summary: ExecutedWriteSummary) + extends WriterCommitMessage + +/** + * Wrapper class for the metrics of writing data out. + * + * @param updatedPartitions + * the partitions updated during writing data out. Only valid for dynamic partition. + * @param stats + * one `WriteTaskStats` object for every `WriteJobStatsTracker` that the job had. + */ +case class ExecutedWriteSummary(updatedPartitions: Set[String], stats: Seq[WriteTaskStats]) diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala new file mode 100644 index 000000000000..cbdf13159c08 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -0,0 +1,466 @@ +/* + * 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.datasources + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} +import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.{ProjectExec, SortExec, SparkPlan, SQLExecution, UnsafeExternalRowSorter} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.{SerializableConfiguration, Utils} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileAlreadyExistsException, Path} +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import java.util.{Date, UUID} + +/** A helper object for writing FileFormat data out to a location. */ +object FileFormatWriter extends Logging { + + /** Describes how output files should be placed in the filesystem. */ + case class OutputSpec( + outputPath: String, + customPartitionLocations: Map[TablePartitionSpec, String], + outputColumns: Seq[Attribute]) + + /** Describes how concurrent output writers should be executed. */ + case class ConcurrentOutputWriterSpec( + maxWriters: Int, + createSorter: () => UnsafeExternalRowSorter) + + /** + * A variable used in tests to check whether the output ordering of the query matches the required + * ordering of the write command. + */ + private[sql] var outputOrderingMatched: Boolean = false + + /** A variable used in tests to check the final executed plan. */ + private[sql] var executedPlan: Option[SparkPlan] = None + + // scalastyle:off argcount + /** + * Basic work flow of this command is: + * 1. Driver side setup, including output committer initialization and data source specific + * preparation work for the write job to be issued. 2. Issues a write job consists of one or + * more executor side tasks, each of which writes all rows within an RDD partition. 3. If no + * exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. 4. If all tasks are + * committed, commit the job, otherwise aborts the job; If any exception is thrown during job + * commitment, also aborts the job. 5. If the job is successfully committed, perform + * post-commit operations such as processing statistics. + * @return + * The set of all partition paths that were updated during this write job. + */ + def write( + sparkSession: SparkSession, + plan: SparkPlan, + fileFormat: FileFormat, + committer: FileCommitProtocol, + outputSpec: OutputSpec, + hadoopConf: Configuration, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + statsTrackers: Seq[WriteJobStatsTracker], + options: Map[String, String], + numStaticPartitionCols: Int = 0): Set[String] = { + require(partitionColumns.size >= numStaticPartitionCols) + + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[InternalRow]) + FileOutputFormat.setOutputPath(job, new Path(outputSpec.outputPath)) + + val partitionSet = AttributeSet(partitionColumns) + // cleanup the internal metadata information of + // the file source metadata attribute if any before write out + val finalOutputSpec = outputSpec.copy(outputColumns = outputSpec.outputColumns + .map(FileSourceMetadataAttribute.cleanupFileSourceMetadataInformation)) + val dataColumns = finalOutputSpec.outputColumns.filterNot(partitionSet.contains) + + val writerBucketSpec = V1WritesUtils.getWriterBucketSpec(bucketSpec, dataColumns, options) + val sortColumns = V1WritesUtils.getBucketSortColumns(bucketSpec, dataColumns) + + val caseInsensitiveOptions = CaseInsensitiveMap(options) + + val dataSchema = dataColumns.toStructType + DataSourceUtils.verifySchema(fileFormat, dataSchema) + DataSourceUtils.checkFieldNames(fileFormat, dataSchema) + // Note: prepareWrite has side effect. It sets "job". + val outputWriterFactory = + fileFormat.prepareWrite(sparkSession, job, caseInsensitiveOptions, dataSchema) + + val description = new WriteJobDescription( + uuid = UUID.randomUUID.toString, + serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), + outputWriterFactory = outputWriterFactory, + allColumns = finalOutputSpec.outputColumns, + dataColumns = dataColumns, + partitionColumns = partitionColumns, + bucketSpec = writerBucketSpec, + path = finalOutputSpec.outputPath, + customPartitionLocations = finalOutputSpec.customPartitionLocations, + maxRecordsPerFile = caseInsensitiveOptions + .get("maxRecordsPerFile") + .map(_.toLong) + .getOrElse(sparkSession.sessionState.conf.maxRecordsPerFile), + timeZoneId = caseInsensitiveOptions + .get(DateTimeUtils.TIMEZONE_OPTION) + .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone), + statsTrackers = statsTrackers + ) + + // We should first sort by dynamic partition columns, then bucket id, and finally sorting + // columns. + val requiredOrdering = partitionColumns.drop(numStaticPartitionCols) ++ + writerBucketSpec.map(_.bucketIdExpression) ++ sortColumns + val writeFilesOpt = V1WritesUtils.getWriteFilesOpt(plan) + + // SPARK-40588: when planned writing is disabled and AQE is enabled, + // plan contains an AdaptiveSparkPlanExec, which does not know + // its final plan's ordering, so we have to materialize that plan first + // it is fine to use plan further down as the final plan is cached in that plan + def materializeAdaptiveSparkPlan(plan: SparkPlan): SparkPlan = plan match { + case a: AdaptiveSparkPlanExec => a.finalPhysicalPlan + case p: SparkPlan => p.withNewChildren(p.children.map(materializeAdaptiveSparkPlan)) + } + + // the sort order doesn't matter + val actualOrdering = writeFilesOpt + .map(_.child) + .getOrElse(materializeAdaptiveSparkPlan(plan)) + .outputOrdering + val orderingMatched = V1WritesUtils.isOrderingMatched(requiredOrdering, actualOrdering) + + SQLExecution.checkSQLExecutionId(sparkSession) + + // propagate the description UUID into the jobs, so that committers + // get an ID guaranteed to be unique. + job.getConfiguration.set("spark.sql.sources.writeJobUUID", description.uuid) + + // When `PLANNED_WRITE_ENABLED` is true, the optimizer rule V1Writes will add logical sort + // operator based on the required ordering of the V1 write command. So the output + // ordering of the physical plan should always match the required ordering. Here + // we set the variable to verify this behavior in tests. + // There are two cases where FileFormatWriter still needs to add physical sort: + // 1) When the planned write config is disabled. + // 2) When the concurrent writers are enabled (in this case the required ordering of a + // V1 write command will be empty). + if (Utils.isTesting) outputOrderingMatched = orderingMatched + + if (writeFilesOpt.isDefined) { + // build `WriteFilesSpec` for `WriteFiles` + val concurrentOutputWriterSpecFunc = (plan: SparkPlan) => { + val sortPlan = createSortPlan(plan, requiredOrdering, outputSpec) + createConcurrentOutputWriterSpec(sparkSession, sortPlan, sortColumns) + } + val writeSpec = WriteFilesSpec( + description = description, + committer = committer, + concurrentOutputWriterSpecFunc = concurrentOutputWriterSpecFunc + ) + executeWrite(sparkSession, plan, writeSpec, job) + } else { + executeWrite( + sparkSession, + plan, + job, + description, + committer, + outputSpec, + requiredOrdering, + partitionColumns, + sortColumns, + orderingMatched) + } + } + // scalastyle:on argcount + + private def executeWrite( + sparkSession: SparkSession, + plan: SparkPlan, + job: Job, + description: WriteJobDescription, + committer: FileCommitProtocol, + outputSpec: OutputSpec, + requiredOrdering: Seq[Expression], + partitionColumns: Seq[Attribute], + sortColumns: Seq[Attribute], + orderingMatched: Boolean): Set[String] = { + val projectList = V1WritesUtils.convertEmptyToNull(plan.output, partitionColumns) + val empty2NullPlan = if (projectList.nonEmpty) ProjectExec(projectList, plan) else plan + + writeAndCommit(job, description, committer) { + val (planToExecute, concurrentOutputWriterSpec) = if (orderingMatched) { + (empty2NullPlan, None) + } else { + val sortPlan = createSortPlan(empty2NullPlan, requiredOrdering, outputSpec) + val concurrentOutputWriterSpec = + createConcurrentOutputWriterSpec(sparkSession, sortPlan, sortColumns) + if (concurrentOutputWriterSpec.isDefined) { + (empty2NullPlan, concurrentOutputWriterSpec) + } else { + (sortPlan, concurrentOutputWriterSpec) + } + } + + // In testing, this is the only way to get hold of the actually executed plan written to file + if (Utils.isTesting) executedPlan = Some(planToExecute) + + val rdd = planToExecute.execute() + + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + val rddWithNonEmptyPartitions = if (rdd.partitions.length == 0) { + sparkSession.sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + rdd + } + + val jobTrackerID = SparkHadoopWriterUtils.createJobTrackerID(new Date()) + val ret = new Array[WriteTaskResult](rddWithNonEmptyPartitions.partitions.length) + sparkSession.sparkContext.runJob( + rddWithNonEmptyPartitions, + (taskContext: TaskContext, iter: Iterator[InternalRow]) => { + executeTask( + description = description, + jobTrackerID = jobTrackerID, + sparkStageId = taskContext.stageId(), + sparkPartitionId = taskContext.partitionId(), + sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, + committer, + iterator = iter, + concurrentOutputWriterSpec = concurrentOutputWriterSpec + ) + }, + rddWithNonEmptyPartitions.partitions.indices, + (index, res: WriteTaskResult) => { + committer.onTaskCommit(res.commitMsg) + ret(index) = res + } + ) + ret + } + } + + private def writeAndCommit( + job: Job, + description: WriteJobDescription, + committer: FileCommitProtocol)(f: => Array[WriteTaskResult]): Set[String] = { + // This call shouldn't be put into the `try` block below because it only initializes and + // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. + committer.setupJob(job) + try { + val ret = f + val commitMsgs = ret.map(_.commitMsg) + + logInfo(s"Start to commit write Job ${description.uuid}.") + val (_, duration) = Utils.timeTakenMs(committer.commitJob(job, commitMsgs)) + logInfo(s"Write Job ${description.uuid} committed. Elapsed time: $duration ms.") + + processStats(description.statsTrackers, ret.map(_.summary.stats), duration) + logInfo(s"Finished processing stats for write job ${description.uuid}.") + + // return a set of all the partition paths that were updated during this job + ret.map(_.summary.updatedPartitions).reduceOption(_ ++ _).getOrElse(Set.empty) + } catch { + case cause: Throwable => + logError(s"Aborting job ${description.uuid}.", cause) + committer.abortJob(job) + throw cause + } + } + + /** Write files using [[SparkPlan.executeWrite]] */ + private def executeWrite( + session: SparkSession, + planForWrites: SparkPlan, + writeFilesSpec: WriteFilesSpec, + job: Job): Set[String] = { + val committer = writeFilesSpec.committer + val description = writeFilesSpec.description + + // In testing, this is the only way to get hold of the actually executed plan written to file + if (Utils.isTesting) executedPlan = Some(planForWrites) + + writeAndCommit(job, description, committer) { + val rdd = planForWrites.executeWrite(writeFilesSpec) + val ret = new Array[WriteTaskResult](rdd.partitions.length) + session.sparkContext.runJob( + rdd, + (context: TaskContext, iter: Iterator[WriterCommitMessage]) => { + assert(iter.hasNext) + val commitMessage = iter.next() + assert(!iter.hasNext) + commitMessage + }, + rdd.partitions.indices, + (index, res: WriterCommitMessage) => { + assert(res.isInstanceOf[WriteTaskResult]) + val writeTaskResult = res.asInstanceOf[WriteTaskResult] + committer.onTaskCommit(writeTaskResult.commitMsg) + ret(index) = writeTaskResult + } + ) + ret + } + } + + private def createSortPlan( + plan: SparkPlan, + requiredOrdering: Seq[Expression], + outputSpec: OutputSpec): SortExec = { + // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and + // the physical plan may have different attribute ids due to optimizer removing some + // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. + val orderingExpr = + bindReferences(requiredOrdering.map(SortOrder(_, Ascending)), outputSpec.outputColumns) + SortExec(orderingExpr, global = false, child = plan) + } + + private def createConcurrentOutputWriterSpec( + sparkSession: SparkSession, + sortPlan: SortExec, + sortColumns: Seq[Attribute]): Option[ConcurrentOutputWriterSpec] = { + val maxWriters = sparkSession.sessionState.conf.maxConcurrentOutputFileWriters + val concurrentWritersEnabled = maxWriters > 0 && sortColumns.isEmpty + if (concurrentWritersEnabled) { + Some(ConcurrentOutputWriterSpec(maxWriters, () => sortPlan.createSorter())) + } else { + None + } + } + + /** Writes data out in a single Spark task. */ + private[spark] def executeTask( + description: WriteJobDescription, + jobTrackerID: String, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + committer: FileCommitProtocol, + iterator: Iterator[InternalRow], + concurrentOutputWriterSpec: Option[ConcurrentOutputWriterSpec]): WriteTaskResult = { + + val jobId = SparkHadoopWriterUtils.createJobID(jobTrackerID, sparkStageId) + val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) + val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) + + // Set up the attempt context required to use in the output committer. + val taskAttemptContext: TaskAttemptContext = { + // Set up the configuration object + val hadoopConf = description.serializableHadoopConf.value + hadoopConf.set("mapreduce.job.id", jobId.toString) + hadoopConf.set("mapreduce.task.id", taskAttemptId.getTaskID.toString) + hadoopConf.set("mapreduce.task.attempt.id", taskAttemptId.toString) + hadoopConf.setBoolean("mapreduce.task.ismap", true) + hadoopConf.setInt("mapreduce.task.partition", 0) + + new TaskAttemptContextImpl(hadoopConf, taskAttemptId) + } + + committer.setupTask(taskAttemptContext) + + val dataWriter = + if (sparkPartitionId != 0 && !iterator.hasNext) { + // In case of empty job, leave first partition to save meta for file format like parquet. + new EmptyDirectoryDataWriter(description, taskAttemptContext, committer) + } else if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { + new SingleDirectoryDataWriter(description, taskAttemptContext, committer) + } else { + concurrentOutputWriterSpec match { + case Some(spec) => + new DynamicPartitionDataConcurrentWriter( + description, + taskAttemptContext, + committer, + spec) + case _ => + new DynamicPartitionDataSingleWriter(description, taskAttemptContext, committer) + } + } + + try { + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out and commit the task. + dataWriter.writeWithIterator(iterator) + dataWriter.commit() + })( + catchBlock = { + // If there is an error, abort the task + dataWriter.abort() + logError(s"Job $jobId aborted.") + }, + finallyBlock = { + dataWriter.close() + }) + } catch { + case e: FetchFailedException => + throw e + case f: FileAlreadyExistsException if SQLConf.get.fastFailFileFormatOutput => + // If any output file to write already exists, it does not make sense to re-run this task. + // We throw the exception and let Executor throw ExceptionFailure to abort the job. + throw new TaskOutputFileAlreadyExistException(f) + case t: Throwable => + throw QueryExecutionErrors.taskFailedWhileWritingRowsError(description.path, t) + } + } + + /** + * For every registered [[WriteJobStatsTracker]], call `processStats()` on it, passing it the + * corresponding [[WriteTaskStats]] from all executors. + */ + private[datasources] def processStats( + statsTrackers: Seq[WriteJobStatsTracker], + statsPerTask: Seq[Seq[WriteTaskStats]], + jobCommitDuration: Long): Unit = { + + val numStatsTrackers = statsTrackers.length + assert( + statsPerTask.forall(_.length == numStatsTrackers), + s"""Every WriteTask should have produced one `WriteTaskStats` object for every tracker. + |There are $numStatsTrackers statsTrackers, but some task returned + |${statsPerTask.find(_.length != numStatsTrackers).get.length} results instead. + """.stripMargin + ) + + val statsPerTracker = if (statsPerTask.nonEmpty) { + statsPerTask.transpose + } else { + statsTrackers.map(_ => Seq.empty) + } + + statsTrackers.zip(statsPerTracker).foreach { + case (statsTracker, stats) => statsTracker.processStats(stats, jobCommitDuration) + } + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala new file mode 100644 index 000000000000..0994bb6e0ff1 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -0,0 +1,257 @@ +/* + * 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.datasources.orc + +import org.apache.spark.TaskContext +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.{SerializableConfiguration, Utils} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcUtils => _, _} +import org.apache.orc.OrcConf.COMPRESS +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce._ + +import java.io._ + +/** New ORC File Format based on Apache ORC. */ +class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable { + + override def shortName(): String = "orc" + + override def toString: String = "ORC" + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[OrcFileFormat] + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + OrcUtils.inferSchema(sparkSession, files, options) + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + + val conf = job.getConfiguration + + conf.set(COMPRESS.getAttribute, orcOptions.compressionCodec) + + conf + .asInstanceOf[JobConf] + .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) + + val batchSize = sparkSession.sessionState.conf.orcVectorizedWriterBatchSize + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new OrcOutputWriter(path, dataSchema, context, batchSize) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(COMPRESS.getAttribute) + OrcUtils.extensionsForCompressionCodecNames.getOrElse(name, "") + } + + compressionExtension + ".orc" + } + } + } + + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + conf.orcVectorizedReaderEnabled && + schema.forall( + s => + OrcUtils.supportColumnarReads( + s.dataType, + sparkSession.sessionState.conf.orcVectorizedReaderNestedColumnEnabled)) + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + + /** + * Build the reader. + * + * @note + * It is required to pass FileFormat.OPTION_RETURNING_BATCH in options, to indicate whether the + * reader should return row or columnar output. If the caller can handle both, pass + * FileFormat.OPTION_RETURNING_BATCH -> supportBatch(sparkSession, + * StructType(requiredSchema.fields ++ partitionSchema.fields)) as the option. It should be set + * to "true" only if this reader can support it. + */ + override def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + + val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) + val sqlConf = sparkSession.sessionState.conf + val capacity = sqlConf.orcVectorizedReaderBatchSize + + // Should always be set by FileSourceScanExec creating this. + // Check conf before checking option, to allow working around an issue by changing conf. + val enableVectorizedReader = sqlConf.orcVectorizedReaderEnabled && + options + .get(FileFormat.OPTION_RETURNING_BATCH) + .getOrElse { + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for OrcFileFormat. " + + "To workaround this issue, set spark.sql.orc.enableVectorizedReader=false.") + } + .equals("true") + if (enableVectorizedReader) { + // If the passed option said that we are to return batches, we need to also be able to + // do this based on config and resultSchema. + assert(supportBatch(sparkSession, resultSchema)) + } + + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis) + + val broadcastedConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + val orcFilterPushDown = sparkSession.sessionState.conf.orcFilterPushDown + + (file: PartitionedFile) => { + val conf = broadcastedConf.value.value + + val filePath = file.toPath + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val orcSchema = + Utils.tryWithResource(OrcFile.createReader(filePath, readerOptions))(_.getSchema) + val resultedColPruneInfo = + OrcUtils.requestedColumnIds(isCaseSensitive, dataSchema, requiredSchema, orcSchema, conf) + + if (resultedColPruneInfo.isEmpty) { + Iterator.empty + } else { + // ORC predicate pushdown + if (orcFilterPushDown && filters.nonEmpty) { + val fileSchema = OrcUtils.toCatalystSchema(orcSchema) + OrcFilters.createFilter(fileSchema, filters).foreach { + f => OrcInputFormat.setSearchArgument(conf, f, fileSchema.fieldNames) + } + } + + val (requestedColIds, canPruneCols) = resultedColPruneInfo.get + val resultSchemaString = OrcUtils.orcResultSchemaString( + canPruneCols, + dataSchema, + resultSchema, + partitionSchema, + conf) + assert( + requestedColIds.length == requiredSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + + val includeColumns = requestedColIds.filter(_ != -1).sorted.mkString(",") + taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, includeColumns) + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + if (enableVectorizedReader) { + val batchReader = new OrcColumnarBatchReader(capacity) + // SPARK-23399 Register a task completion listener first to call `close()` in all cases. + // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) + // after opening a file. + val iter = new RecordReaderIterator(batchReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + val requestedDataColIds = requestedColIds ++ Array.fill(partitionSchema.length)(-1) + val requestedPartitionColIds = + Array.fill(requiredSchema.length)(-1) ++ Range(0, partitionSchema.length) + batchReader.initialize(fileSplit, taskAttemptContext) + batchReader.initBatch( + TypeDescription.fromString(resultSchemaString), + resultSchema.fields, + requestedDataColIds, + requestedPartitionColIds, + file.partitionValues) + + iter.asInstanceOf[Iterator[InternalRow]] + } else { + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val iter = new RecordReaderIterator[OrcStruct](orcRecordReader) + Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + val deserializer = new OrcDeserializer(requiredSchema, requestedColIds) + + if (partitionSchema.length == 0) { + iter.map(value => unsafeProjection(deserializer.deserialize(value))) + } else { + val joinedRow = new JoinedRow() + iter.map( + value => + unsafeProjection(joinedRow(deserializer.deserialize(value), file.partitionValues))) + } + } + } + } + } + + override def supportDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall(f => supportDataType(f.dataType)) + + case ArrayType(elementType, _) => supportDataType(elementType) + + case MapType(keyType, valueType, _) => + supportDataType(keyType) && supportDataType(valueType) + + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) + + case _ => false + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala new file mode 100644 index 000000000000..c44092d71975 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -0,0 +1,515 @@ +/* + * 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.datasources.parquet + +import org.apache.spark.TaskContext +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.parser.LegacyTypeStringParser +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.vectorized.{ConstantColumnVector, OffHeapColumnVector, OnHeapColumnVector} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.types._ +import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapred.FileSplit +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.predicate.FilterApi +import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS +import org.apache.parquet.hadoop._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.{Failure, Try} + +class ParquetFileFormat extends FileFormat with DataSourceRegister with Logging with Serializable { + + override def shortName(): String = "parquet" + + override def toString: String = "Parquet" + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[ParquetFileFormat] + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val sqlConf = sparkSession.sessionState.conf + val parquetOptions = new ParquetOptions(options, sqlConf) + ParquetUtils.prepareWrite(sqlConf, job, dataSchema, parquetOptions) + } + + override def inferSchema( + sparkSession: SparkSession, + parameters: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + ParquetUtils.inferSchema(sparkSession, parameters, files) + } + + /** Returns whether the reader can return the rows as batch or not. */ + override def supportBatch(sparkSession: SparkSession, schema: StructType): Boolean = { + val conf = sparkSession.sessionState.conf + ParquetUtils.isBatchReadSupportedForSchema(conf, schema) + } + + override def vectorTypes( + requiredSchema: StructType, + partitionSchema: StructType, + sqlConf: SQLConf): Option[Seq[String]] = { + Option( + Seq.fill(requiredSchema.fields.length)( + if (!sqlConf.offHeapColumnVectorEnabled) { + classOf[OnHeapColumnVector].getName + } else { + classOf[OffHeapColumnVector].getName + } + ) ++ Seq.fill(partitionSchema.fields.length)(classOf[ConstantColumnVector].getName)) + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + true + } + + /** + * Build the reader. + * + * @note + * It is required to pass FileFormat.OPTION_RETURNING_BATCH in options, to indicate whether the + * reader should return row or columnar output. If the caller can handle both, pass + * FileFormat.OPTION_RETURNING_BATCH -> supportBatch(sparkSession, + * StructType(requiredSchema.fields ++ partitionSchema.fields)) as the option. It should be set + * to "true" only if this reader can support it. + */ + override def buildReaderWithPartitionValues( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): (PartitionedFile) => Iterator[InternalRow] = { + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, requiredSchema.json) + hadoopConf.set(ParquetWriteSupport.SPARK_ROW_SCHEMA, requiredSchema.json) + hadoopConf.set( + SQLConf.SESSION_LOCAL_TIMEZONE.key, + sparkSession.sessionState.conf.sessionLocalTimeZone) + hadoopConf.setBoolean( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, + sparkSession.sessionState.conf.nestedSchemaPruningEnabled) + hadoopConf.setBoolean( + SQLConf.CASE_SENSITIVE.key, + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + // Sets flags for `ParquetToSparkSchemaConverter` + hadoopConf.setBoolean( + SQLConf.PARQUET_BINARY_AS_STRING.key, + sparkSession.sessionState.conf.isParquetBinaryAsString) + hadoopConf.setBoolean( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sparkSession.sessionState.conf.isParquetINT96AsTimestamp) + hadoopConf.setBoolean( + SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED.key, + sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled) + hadoopConf.setBoolean( + SQLConf.LEGACY_PARQUET_NANOS_AS_LONG.key, + sparkSession.sessionState.conf.legacyParquetNanosAsLong) + + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + + // TODO: if you move this into the closure it reverts to the default values. + // If true, enable using the custom RecordReader for parquet. This only works for + // a subset of the types (no complex types). + val resultSchema = StructType(partitionSchema.fields ++ requiredSchema.fields) + val sqlConf = sparkSession.sessionState.conf + val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + val enableVectorizedReader: Boolean = + ParquetUtils.isBatchReadSupportedForSchema(sqlConf, resultSchema) + val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled + val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion + val capacity = sqlConf.parquetVectorizedReaderBatchSize + val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown + val pushDownDate = sqlConf.parquetFilterPushDownDate + val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp + val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal + val pushDownStringPredicate = sqlConf.parquetFilterPushDownStringPredicate + val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold + val isCaseSensitive = sqlConf.caseSensitiveAnalysis + val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead + val int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead + + // Should always be set by FileSourceScanExec creating this. + // Check conf before checking option, to allow working around an issue by changing conf. + val returningBatch = sparkSession.sessionState.conf.parquetVectorizedReaderEnabled && + options + .get(FileFormat.OPTION_RETURNING_BATCH) + .getOrElse { + throw new IllegalArgumentException( + "OPTION_RETURNING_BATCH should always be set for ParquetFileFormat. " + + "To workaround this issue, set spark.sql.parquet.enableVectorizedReader=false.") + } + .equals("true") + if (returningBatch) { + // If the passed option said that we are to return batches, we need to also be able to + // do this based on config and resultSchema. + assert(supportBatch(sparkSession, resultSchema)) + } + + (file: PartitionedFile) => { + assert(file.partitionValues.numFields == partitionSchema.size) + + val filePath = file.toPath + val split = new FileSplit(filePath, file.start, file.length, Array.empty[String]) + + val sharedConf = broadcastedHadoopConf.value.value + + lazy val footerFileMetaData = + ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData + val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + datetimeRebaseModeInRead) + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters( + parquetSchema, + pushDownDate, + pushDownTimestamp, + pushDownDecimal, + pushDownStringPredicate, + pushDownInFilterThreshold, + isCaseSensitive, + datetimeRebaseSpec) + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(parquetFilters.createFilter(_)) + .reduceOption(FilterApi.and) + } else { + None + } + + // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps' + // *only* if the file was created by something other than "parquet-mr", so check the actual + // writer here for this file. We have to do this per-file, as each file in the table may + // have different writers. + // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads. + def isCreatedByParquetMr: Boolean = + footerFileMetaData.getCreatedBy().startsWith("parquet-mr") + + val convertTz = + if (timestampConversion && !isCreatedByParquetMr) { + Some(DateTimeUtils.getZoneId(sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val int96RebaseSpec = DataSourceUtils.int96RebaseSpec( + footerFileMetaData.getKeyValueMetaData.get, + int96RebaseModeInRead) + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = + new TaskAttemptContextImpl(broadcastedHadoopConf.value.value, attemptId) + + // Try to push down filters when filter push-down is enabled. + // Notice: This push-down is RowGroups level, not individual records. + if (pushed.isDefined) { + ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) + } + val taskContext = Option(TaskContext.get()) + if (enableVectorizedReader) { + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, + datetimeRebaseSpec.mode.toString, + datetimeRebaseSpec.timeZone, + int96RebaseSpec.mode.toString, + int96RebaseSpec.timeZone, + enableOffHeapColumnVector && taskContext.isDefined, + capacity + ) + // SPARK-37089: We cannot register a task completion listener to close this iterator here + // because downstream exec nodes have already registered their listeners. Since listeners + // are executed in reverse order of registration, a listener registered here would close the + // iterator while downstream exec nodes are still running. When off-heap column vectors are + // enabled, this can cause a use-after-free bug leading to a segfault. + // + // Instead, we use FileScanRDD's task completion listener to close this iterator. + val iter = new RecordReaderIterator(vectorizedReader) + try { + vectorizedReader.initialize(split, hadoopAttemptContext) + logDebug(s"Appending $partitionSchema ${file.partitionValues}") + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + if (returningBatch) { + vectorizedReader.enableReturningBatches() + } + + // UnsafeRowParquetRecordReader appends the columns internally to avoid another copy. + iter.asInstanceOf[Iterator[InternalRow]] + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } else { + logDebug(s"Falling back to parquet-mr") + // ParquetRecordReader returns InternalRow + val readSupport = new ParquetReadSupport( + convertTz, + enableVectorizedReader = false, + datetimeRebaseSpec, + int96RebaseSpec) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[InternalRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[InternalRow](readSupport) + } + val readerWithRowIndexes = + ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader, requiredSchema) + val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes) + try { + readerWithRowIndexes.initialize(split, hadoopAttemptContext) + + val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes + val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + + if (partitionSchema.length == 0) { + // There is no partition columns + iter.map(unsafeProjection) + } else { + val joinedRow = new JoinedRow() + iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues))) + } + } catch { + case e: Throwable => + // SPARK-23457: In case there is an exception in initialization, close the iterator to + // avoid leaking resources. + iter.close() + throw e + } + } + } + } + + override def supportDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall(f => supportDataType(f.dataType)) + + case ArrayType(elementType, _) => supportDataType(elementType) + + case MapType(keyType, valueType, _) => + supportDataType(keyType) && supportDataType(valueType) + + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) + + case _ => false + } +} + +object ParquetFileFormat extends Logging { + private[parquet] def readSchema( + footers: Seq[Footer], + sparkSession: SparkSession): Option[StructType] = { + + val converter = new ParquetToSparkSchemaConverter( + sparkSession.sessionState.conf.isParquetBinaryAsString, + sparkSession.sessionState.conf.isParquetINT96AsTimestamp, + inferTimestampNTZ = sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled, + nanosAsLong = sparkSession.sessionState.conf.legacyParquetNanosAsLong + ) + + val seen = mutable.HashSet[String]() + val finalSchemas: Seq[StructType] = footers.flatMap { + footer => + val metadata = footer.getParquetMetadata.getFileMetaData + val serializedSchema = metadata.getKeyValueMetaData.asScala.toMap + .get(ParquetReadSupport.SPARK_METADATA_KEY) + if (serializedSchema.isEmpty) { + // Falls back to Parquet schema if no Spark SQL schema found. + Some(converter.convert(metadata.getSchema)) + } else if (!seen.contains(serializedSchema.get)) { + seen += serializedSchema.get + + // Don't throw even if we failed to parse the serialized Spark schema. Just fallback to + // whatever is available. + Some(Try(DataType.fromJson(serializedSchema.get)) + .recover { + case _: Throwable => + logInfo( + "Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + + "falling back to the deprecated DataType.fromCaseClassString parser.") + LegacyTypeStringParser.parseString(serializedSchema.get) + } + .recover { + case cause: Throwable => + logWarning( + s"""Failed to parse serialized Spark schema in Parquet key-value metadata: + |\t$serializedSchema + """.stripMargin, + cause + ) + } + .map(_.asInstanceOf[StructType]) + .getOrElse { + // Falls back to Parquet schema if Spark SQL schema can't be parsed. + converter.convert(metadata.getSchema) + }) + } else { + None + } + } + + finalSchemas.reduceOption { + (left, right) => + try left.merge(right) + catch { + case e: Throwable => + throw QueryExecutionErrors.failedToMergeIncompatibleSchemasError(left, right, e) + } + } + } + + /** + * Reads Parquet footers in multi-threaded manner. If the config + * "spark.sql.files.ignoreCorruptFiles" is set to true, we will ignore the corrupted files when + * reading footers. + */ + private[parquet] def readParquetFootersInParallel( + conf: Configuration, + partFiles: Seq[FileStatus], + ignoreCorruptFiles: Boolean): Seq[Footer] = { + ThreadUtils + .parmap(partFiles, "readingParquetFooters", 8) { + currentFile => + try { + // Skips row group information since we only need the schema. + // ParquetFileReader.readFooter throws RuntimeException, instead of IOException, + // when it can't read the footer. + Some( + new Footer( + currentFile.getPath(), + ParquetFooterReader.readFooter(conf, currentFile, SKIP_ROW_GROUPS))) + } catch { + case e: RuntimeException => + if (ignoreCorruptFiles) { + logWarning(s"Skipped the footer in the corrupted file: $currentFile", e) + None + } else { + throw QueryExecutionErrors.cannotReadFooterForFileError(currentFile.getPath, e) + } + } + } + .flatten + } + + /** + * Figures out a merged Parquet schema with a distributed Spark job. + * + * Note that locality is not taken into consideration here because: + * + * 1. For a single Parquet part-file, in most cases the footer only resides in the last block of + * that file. Thus we only need to retrieve the location of the last block. However, Hadoop + * `FileSystem` only provides API to retrieve locations of all blocks, which can be + * potentially expensive. + * + * 2. This optimization is mainly useful for S3, where file metadata operations can be pretty + * slow. And basically locality is not available when using S3 (you can't run computation on S3 + * nodes). + */ + def mergeSchemasInParallel( + parameters: Map[String, String], + filesToTouch: Seq[FileStatus], + sparkSession: SparkSession): Option[StructType] = { + val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString + val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp + val inferTimestampNTZ = sparkSession.sessionState.conf.parquetInferTimestampNTZEnabled + val nanosAsLong = sparkSession.sessionState.conf.legacyParquetNanosAsLong + + val reader = (files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) => { + // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` + val converter = new ParquetToSparkSchemaConverter( + assumeBinaryIsString = assumeBinaryIsString, + assumeInt96IsTimestamp = assumeInt96IsTimestamp, + inferTimestampNTZ = inferTimestampNTZ, + nanosAsLong = nanosAsLong) + + readParquetFootersInParallel(conf, files, ignoreCorruptFiles) + .map(ParquetFileFormat.readSchemaFromFooter(_, converter)) + } + + SchemaMergeUtils.mergeSchemasInParallel(sparkSession, parameters, filesToTouch, reader) + } + + /** + * Reads Spark SQL schema from a Parquet footer. If a valid serialized Spark SQL schema string can + * be found in the file metadata, returns the deserialized [[StructType]], otherwise, returns a + * [[StructType]] converted from the [[org.apache.parquet.schema.MessageType]] stored in this + * footer. + */ + def readSchemaFromFooter(footer: Footer, converter: ParquetToSparkSchemaConverter): StructType = { + val fileMetaData = footer.getParquetMetadata.getFileMetaData + fileMetaData.getKeyValueMetaData.asScala.toMap + .get(ParquetReadSupport.SPARK_METADATA_KEY) + .flatMap(deserializeSchemaString) + .getOrElse(converter.convert(fileMetaData.getSchema)) + } + + private def deserializeSchemaString(schemaString: String): Option[StructType] = { + // Tries to deserialize the schema string as JSON first, then falls back to the case class + // string parser (data generated by older versions of Spark SQL uses this format). + Try(DataType.fromJson(schemaString).asInstanceOf[StructType]) + .recover { + case _: Throwable => + logInfo( + "Serialized Spark schema in Parquet key-value metadata is not in JSON format, " + + "falling back to the deprecated DataType.fromCaseClassString parser.") + LegacyTypeStringParser.parseString(schemaString).asInstanceOf[StructType] + } + .recoverWith { + case cause: Throwable => + logWarning( + "Failed to parse and ignored serialized Spark schema in " + + s"Parquet key-value metadata:\n\t$schemaString", + cause) + Failure(cause) + } + .toOption + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala.deprecated b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala.deprecated new file mode 100644 index 000000000000..d43331d57c47 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala.deprecated @@ -0,0 +1,243 @@ +/* + * 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.datasources.v2 + +import com.google.common.base.Objects + +import org.apache.spark.SparkException +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.QueryPlan +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.connector.catalog.Table +import org.apache.spark.sql.connector.read._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Physical plan node for scanning a batch of data from a data source v2. + */ +case class BatchScanExec( + output: Seq[AttributeReference], + @transient scan: Scan, + runtimeFilters: Seq[Expression], + keyGroupedPartitioning: Option[Seq[Expression]] = None, + ordering: Option[Seq[SortOrder]] = None, + @transient table: Table, + commonPartitionValues: Option[Seq[(InternalRow, Int)]] = None, + applyPartialClustering: Boolean = false, + replicatePartitions: Boolean = false) extends DataSourceV2ScanExecBase { + + @transient lazy val batch = if (scan == null) null else scan.toBatch + + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: BatchScanExec => + this.batch != null && this.batch == other.batch && + this.runtimeFilters == other.runtimeFilters && + this.commonPartitionValues == other.commonPartitionValues && + this.replicatePartitions == other.replicatePartitions && + this.applyPartialClustering == other.applyPartialClustering + case _ => + false + } + + override def hashCode(): Int = Objects.hashCode(batch, runtimeFilters) + + @transient override lazy val inputPartitions: Seq[InputPartition] = batch.planInputPartitions() + + @transient private lazy val filteredPartitions: Seq[Seq[InputPartition]] = { + val dataSourceFilters = runtimeFilters.flatMap { + case DynamicPruningExpression(e) => DataSourceV2Strategy.translateRuntimeFilterV2(e) + case _ => None + } + + if (dataSourceFilters.nonEmpty) { + val originalPartitioning = outputPartitioning + + // the cast is safe as runtime filters are only assigned if the scan can be filtered + val filterableScan = scan.asInstanceOf[SupportsRuntimeV2Filtering] + filterableScan.filter(dataSourceFilters.toArray) + + // call toBatch again to get filtered partitions + val newPartitions = scan.toBatch.planInputPartitions() + + originalPartitioning match { + case p: KeyGroupedPartitioning => + if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) { + throw new SparkException("Data source must have preserved the original partitioning " + + "during runtime filtering: not all partitions implement HasPartitionKey after " + + "filtering") + } + val newPartitionValues = newPartitions.map(partition => + InternalRowComparableWrapper(partition.asInstanceOf[HasPartitionKey], p.expressions)) + .toSet + val oldPartitionValues = p.partitionValues + .map(partition => InternalRowComparableWrapper(partition, p.expressions)).toSet + // We require the new number of partition values to be equal or less than the old number + // of partition values here. In the case of less than, empty partitions will be added for + // those missing values that are not present in the new input partitions. + if (oldPartitionValues.size < newPartitionValues.size) { + throw new SparkException("During runtime filtering, data source must either report " + + "the same number of partition values, or a subset of partition values from the " + + s"original. Before: ${oldPartitionValues.size} partition values. " + + s"After: ${newPartitionValues.size} partition values") + } + + if (!newPartitionValues.forall(oldPartitionValues.contains)) { + throw new SparkException("During runtime filtering, data source must not report new " + + "partition values that are not present in the original partitioning.") + } + + groupPartitions(newPartitions).get.map(_._2) + + case _ => + // no validation is needed as the data source did not report any specific partitioning + newPartitions.map(Seq(_)) + } + + } else { + partitions + } + } + + override def outputPartitioning: Partitioning = { + super.outputPartitioning match { + case k: KeyGroupedPartitioning if commonPartitionValues.isDefined => + // We allow duplicated partition values if + // `spark.sql.sources.v2.bucketing.partiallyClusteredDistribution.enabled` is true + val newPartValues = commonPartitionValues.get.flatMap { case (partValue, numSplits) => + Seq.fill(numSplits)(partValue) + } + k.copy(numPartitions = newPartValues.length, partitionValues = newPartValues) + case p => p + } + } + + override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() + + override lazy val inputRDD: RDD[InternalRow] = { + val rdd = if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { + // return an empty RDD with 1 partition if dynamic filtering removed the only split + sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + var finalPartitions = filteredPartitions + + outputPartitioning match { + case p: KeyGroupedPartitioning => + if (conf.v2BucketingPushPartValuesEnabled && + conf.v2BucketingPartiallyClusteredDistributionEnabled) { + assert(filteredPartitions.forall(_.size == 1), + "Expect partitions to be not grouped when " + + s"${SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED.key} " + + "is enabled") + + val groupedPartitions = groupPartitions(finalPartitions.map(_.head), true).get + + // This means the input partitions are not grouped by partition values. We'll need to + // check `groupByPartitionValues` and decide whether to group and replicate splits + // within a partition. + if (commonPartitionValues.isDefined && applyPartialClustering) { + // A mapping from the common partition values to how many splits the partition + // should contain. Note this no longer maintain the partition key ordering. + val commonPartValuesMap = commonPartitionValues + .get + .map(t => (InternalRowComparableWrapper(t._1, p.expressions), t._2)) + .toMap + val nestGroupedPartitions = groupedPartitions.map { + case (partValue, splits) => + // `commonPartValuesMap` should contain the part value since it's the super set. + val numSplits = commonPartValuesMap + .get(InternalRowComparableWrapper(partValue, p.expressions)) + assert(numSplits.isDefined, s"Partition value $partValue does not exist in " + + "common partition values from Spark plan") + + val newSplits = if (replicatePartitions) { + // We need to also replicate partitions according to the other side of join + Seq.fill(numSplits.get)(splits) + } else { + // Not grouping by partition values: this could be the side with partially + // clustered distribution. Because of dynamic filtering, we'll need to check if + // the final number of splits of a partition is smaller than the original + // number, and fill with empty splits if so. This is necessary so that both + // sides of a join will have the same number of partitions & splits. + splits.map(Seq(_)).padTo(numSplits.get, Seq.empty) + } + (InternalRowComparableWrapper(partValue, p.expressions), newSplits) + } + + // Now fill missing partition keys with empty partitions + val partitionMapping = nestGroupedPartitions.toMap + finalPartitions = commonPartitionValues.get.flatMap { case (partValue, numSplits) => + // Use empty partition for those partition values that are not present. + partitionMapping.getOrElse( + InternalRowComparableWrapper(partValue, p.expressions), + Seq.fill(numSplits)(Seq.empty)) + } + } else { + val partitionMapping = groupedPartitions.map { case (row, parts) => + InternalRowComparableWrapper(row, p.expressions) -> parts + }.toMap + finalPartitions = p.partitionValues.map { partValue => + // Use empty partition for those partition values that are not present + partitionMapping.getOrElse( + InternalRowComparableWrapper(partValue, p.expressions), Seq.empty) + } + } + } else { + val partitionMapping = finalPartitions.map { parts => + val row = parts.head.asInstanceOf[HasPartitionKey].partitionKey() + InternalRowComparableWrapper(row, p.expressions) -> parts + }.toMap + finalPartitions = p.partitionValues.map { partValue => + // Use empty partition for those partition values that are not present + partitionMapping.getOrElse( + InternalRowComparableWrapper(partValue, p.expressions), Seq.empty) + } + } + + case _ => + } + + new DataSourceRDD( + sparkContext, finalPartitions, readerFactory, supportsColumnar, customMetrics) + } + postDriverMetrics() + rdd + } + + override def doCanonicalize(): BatchScanExec = { + this.copy( + output = output.map(QueryPlan.normalizeExpressions(_, output)), + runtimeFilters = QueryPlan.normalizePredicates( + runtimeFilters.filterNot(_ == DynamicPruningExpression(Literal.TrueLiteral)), + output)) + } + + 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 + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala new file mode 100644 index 000000000000..b7f781a0aeb8 --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExecShim.scala @@ -0,0 +1,133 @@ +/* + * 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.datasources.v2 + +import io.glutenproject.GlutenConfig + +import org.apache.spark.SparkException +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 +import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.connector.expressions.aggregate.Aggregation +import org.apache.spark.sql.connector.read.{HasPartitionKey, InputPartition, Scan, SupportsRuntimeFiltering} +import org.apache.spark.sql.connector.read.SupportsRuntimeV2Filtering +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.vectorized.ColumnarBatch + +import com.google.common.base.Objects + +class BatchScanExecShim( + output: Seq[AttributeReference], + @transient scan: Scan, + runtimeFilters: Seq[Expression], + @transient table: Table) + extends BatchScanExec(output, scan, runtimeFilters, table = table) { + + // Note: "metrics" is made transient to avoid sending driver-side metrics to tasks. + @transient override lazy val metrics: Map[String, SQLMetric] = Map() + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + throw new UnsupportedOperationException("Need to implement this method") + } + + override def equals(other: Any): Boolean = other match { + case that: BatchScanExecShim => + (that.canEqual(this)) && super.equals(that) + case _ => false + } + + override def hashCode(): Int = super.hashCode() + + override def canEqual(other: Any): Boolean = other.isInstanceOf[BatchScanExecShim] + + @transient protected lazy val filteredPartitions: Seq[Seq[InputPartition]] = { + val dataSourceFilters = runtimeFilters.flatMap { + case DynamicPruningExpression(e) => DataSourceV2Strategy.translateRuntimeFilterV2(e) + case _ => None + } + + if (dataSourceFilters.nonEmpty) { + val originalPartitioning = outputPartitioning + + // the cast is safe as runtime filters are only assigned if the scan can be filtered + val filterableScan = scan.asInstanceOf[SupportsRuntimeV2Filtering] + filterableScan.filter(dataSourceFilters.toArray) + + // call toBatch again to get filtered partitions + val newPartitions = scan.toBatch.planInputPartitions() + + originalPartitioning match { + case p: KeyGroupedPartitioning => + if (newPartitions.exists(!_.isInstanceOf[HasPartitionKey])) { + throw new SparkException( + "Data source must have preserved the original partitioning " + + "during runtime filtering: not all partitions implement HasPartitionKey after " + + "filtering") + } + val newPartitionValues = newPartitions + .map( + partition => + InternalRowComparableWrapper( + partition.asInstanceOf[HasPartitionKey], + p.expressions)) + .toSet + val oldPartitionValues = p.partitionValues + .map(partition => InternalRowComparableWrapper(partition, p.expressions)) + .toSet + // We require the new number of partition values to be equal or less than the old number + // of partition values here. In the case of less than, empty partitions will be added for + // those missing values that are not present in the new input partitions. + if (oldPartitionValues.size < newPartitionValues.size) { + throw new SparkException( + "During runtime filtering, data source must either report " + + "the same number of partition values, or a subset of partition values from the " + + s"original. Before: ${oldPartitionValues.size} partition values. " + + s"After: ${newPartitionValues.size} partition values") + } + + if (!newPartitionValues.forall(oldPartitionValues.contains)) { + throw new SparkException( + "During runtime filtering, data source must not report new " + + "partition values that are not present in the original partitioning.") + } + + groupPartitions(newPartitions).get.map(_._2) + + case _ => + // no validation is needed as the data source did not report any specific partitioning + newPartitions.map(Seq(_)) + } + + } else { + partitions + } + } + + @transient lazy val pushedAggregate: Option[Aggregation] = { + scan match { + case s: ParquetScan => s.pushedAggregate + case o: OrcScan => o.pushedAggregate + case _ => None + } + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/Spark34Scan.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/Spark34Scan.scala new file mode 100644 index 000000000000..393db5b87cee --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/Spark34Scan.scala @@ -0,0 +1,50 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory, Scan} + +class Spark34Scan extends DataSourceV2ScanExecBase { + + override def scan: Scan = throw new UnsupportedOperationException("Spark34Scan") + + override def ordering: Option[Seq[SortOrder]] = throw new UnsupportedOperationException( + "Spark34Scan") + + override def readerFactory: PartitionReaderFactory = + throw new UnsupportedOperationException("Spark34Scan") + + override def keyGroupedPartitioning: Option[Seq[Expression]] = + throw new UnsupportedOperationException("Spark34Scan") + + override protected def inputPartitions: Seq[InputPartition] = + throw new UnsupportedOperationException("Spark34Scan") + + override def inputRDD: RDD[InternalRow] = throw new UnsupportedOperationException("Spark34Scan") + + override def output: Seq[Attribute] = throw new UnsupportedOperationException("Spark34Scan") + + override def productElement(n: Int): Any = throw new UnsupportedOperationException("Spark34Scan") + + override def productArity: Int = throw new UnsupportedOperationException("Spark34Scan") + + override def canEqual(that: Any): Boolean = throw new UnsupportedOperationException("Spark34Scan") + +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/utils/CatalogUtil.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/utils/CatalogUtil.scala new file mode 100644 index 000000000000..37ae5984e37b --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/execution/datasources/v2/utils/CatalogUtil.scala @@ -0,0 +1,28 @@ +/* + * 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.datasources.v2.utils + +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.connector.expressions.Transform + +object CatalogUtil { + + def convertPartitionTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.TransformHelper + partitions.toSeq.convertTransforms + } +} diff --git a/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala new file mode 100644 index 000000000000..e970cdb67d8a --- /dev/null +++ b/shims/spark34/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.hive.execution + +import io.glutenproject.execution.datasource.GlutenOrcWriterInjects +import io.glutenproject.execution.datasource.GlutenParquetWriterInjects + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SPECULATION_ENABLED +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.orc.OrcOptions +import org.apache.spark.sql.execution.datasources.parquet.ParquetOptions +import org.apache.spark.sql.hive.{HiveInspectors, HiveTableUtil} +import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableJobConf + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.serde2.Serializer +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, StructObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred.{JobConf, Reporter} +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import scala.collection.JavaConverters._ + +/** + * `FileFormat` for writing Hive tables. + * + * TODO: implement the read logic. + */ +class HiveFileFormat(fileSinkConf: FileSinkDesc) + extends FileFormat + with DataSourceRegister + with Logging { + + def this() = this(null) + + override def shortName(): String = "hive" + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + throw QueryExecutionErrors.inferSchemaUnsupportedForHiveError() + } + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val conf = job.getConfiguration + val tableDesc = fileSinkConf.getTableInfo + conf.set("mapred.output.format.class", tableDesc.getOutputFileFormatClassName) + + // When speculation is on and output committer class name contains "Direct", we should warn + // users that they may loss data if they are using a direct output committer. + val speculationEnabled = sparkSession.sparkContext.conf.get(SPECULATION_ENABLED) + val outputCommitterClass = conf.get("mapred.output.committer.class", "") + if (speculationEnabled && outputCommitterClass.contains("Direct")) { + val warningMessage = + s"$outputCommitterClass may be an output committer that writes data directly to " + + "the final location. Because speculation is enabled, this output committer may " + + "cause data loss (see the case in SPARK-10063). If possible, please use an output " + + "committer that does not have this behavior (e.g. FileOutputCommitter)." + logWarning(warningMessage) + } + + // Add table properties from storage handler to hadoopConf, so any custom storage + // handler settings can be set to hadoopConf + HiveTableUtil.configureJobPropertiesForStorageHandler(tableDesc, conf, false) + Utilities.copyTableJobPropertiesToConf(tableDesc, conf) + + // Avoid referencing the outer object. + val fileSinkConfSer = fileSinkConf + val outputFormat = fileSinkConf.tableInfo.getOutputFileFormatClassName + if ("true".equals(sparkSession.sparkContext.getLocalProperty("isNativeAppliable"))) { + val nativeFormat = sparkSession.sparkContext.getLocalProperty("nativeFormat") + val isParquetFormat = nativeFormat.equals("parquet") + val compressionCodec = if (fileSinkConf.compressed) { + // hive related configurations + fileSinkConf.compressCodec + } else if (isParquetFormat) { + val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf) + parquetOptions.compressionCodecClassName + } else { + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + orcOptions.compressionCodec + } + + val nativeConf = if (isParquetFormat) { + logInfo("Use Gluten parquet write for hive") + GlutenParquetWriterInjects.getInstance().nativeConf(options, compressionCodec) + } else { + logInfo("Use Gluten orc write for hive") + GlutenOrcWriterInjects.getInstance().nativeConf(options, compressionCodec) + } + + new OutputWriterFactory { + private val jobConf = new SerializableJobConf(new JobConf(conf)) + @transient private lazy val outputFormat = + jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] + + override def getFileExtension(context: TaskAttemptContext): String = { + Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) + } + + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + if (isParquetFormat) { + GlutenParquetWriterInjects + .getInstance() + .createOutputWriter(path, dataSchema, context, nativeConf); + } else { + GlutenOrcWriterInjects + .getInstance() + .createOutputWriter(path, dataSchema, context, nativeConf); + } + } + } + } else { + new OutputWriterFactory { + private val jobConf = new SerializableJobConf(new JobConf(conf)) + @transient private lazy val outputFormat = + jobConf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef, Writable]] + + override def getFileExtension(context: TaskAttemptContext): String = { + Utilities.getFileExtension(jobConf.value, fileSinkConfSer.getCompressed, outputFormat) + } + + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new HiveOutputWriter(path, fileSinkConfSer, jobConf.value, dataSchema) + } + } + } + } + + override def supportFieldName(name: String): Boolean = { + fileSinkConf.getTableInfo.getOutputFileFormatClassName match { + case "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat" => + !name.matches(".*[ ,;{}()\n\t=].*") + case "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat" => + try { + TypeInfoUtils.getTypeInfoFromTypeString(s"struct<$name:int>") + true + } catch { + case _: IllegalArgumentException => false + } + case _ => true + } + } +} + +class HiveOutputWriter( + val path: String, + fileSinkConf: FileSinkDesc, + jobConf: JobConf, + dataSchema: StructType) + extends OutputWriter + with HiveInspectors { + + private def tableDesc = fileSinkConf.getTableInfo + + private val serializer = { + val serializer = + tableDesc.getDeserializerClass.getConstructor().newInstance().asInstanceOf[Serializer] + serializer.initialize(jobConf, tableDesc.getProperties) + serializer + } + + private val hiveWriter = HiveFileFormatUtils.getHiveRecordWriter( + jobConf, + tableDesc, + serializer.getSerializedClass, + fileSinkConf, + new Path(path), + Reporter.NULL) + + /** + * Since SPARK-30201 ObjectInspectorCopyOption.JAVA change to ObjectInspectorCopyOption.DEFAULT. + * The reason is DEFAULT option can convert `UTF8String` to `Text` with bytes and we can + * compatible with non UTF-8 code bytes during write. + */ + private val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + tableDesc.getDeserializer(jobConf).getObjectInspector, + ObjectInspectorCopyOption.DEFAULT) + .asInstanceOf[StructObjectInspector] + + private val fieldOIs = + standardOI.getAllStructFieldRefs.asScala.map(_.getFieldObjectInspector).toArray + private val dataTypes = dataSchema.map(_.dataType).toArray + private val wrappers = fieldOIs.zip(dataTypes).map { case (f, dt) => wrapperFor(f, dt) } + private val outputData = new Array[Any](fieldOIs.length) + + override def write(row: InternalRow): Unit = { + var i = 0 + while (i < fieldOIs.length) { + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row.get(i, dataTypes(i))) + i += 1 + } + hiveWriter.write(serializer.serialize(outputData, standardOI)) + } + + override def close(): Unit = { + // Seems the boolean value passed into close does not matter. + hiveWriter.close(false) + } +} diff --git a/substrait/substrait-spark/pom.xml b/substrait/substrait-spark/pom.xml index 5a7f37cb0ee5..51fe2fd5b108 100644 --- a/substrait/substrait-spark/pom.xml +++ b/substrait/substrait-spark/pom.xml @@ -15,6 +15,12 @@ Gluten Substrait Spark + + io.glutenproject + ${sparkshim.artifactId} + ${project.version} + compile + org.apache.spark spark-sql_${scala.binary.version} diff --git a/substrait/substrait-spark/src/main/scala/io/substrait/spark/logical/ToSubstraitRel.scala b/substrait/substrait-spark/src/main/scala/io/substrait/spark/logical/ToSubstraitRel.scala index 9a10220f75e9..f8cf3767938a 100644 --- a/substrait/substrait-spark/src/main/scala/io/substrait/spark/logical/ToSubstraitRel.scala +++ b/substrait/substrait-spark/src/main/scala/io/substrait/spark/logical/ToSubstraitRel.scala @@ -251,6 +251,11 @@ class ToSubstraitRel extends AbstractLogicalPlanVisitor with Logging { relation.Sort.builder.addAllSortFields(fields).input(input).build } + override def visitOffset(plan: Offset): relation.Rel = { + throw new UnsupportedOperationException( + s"Unable to convert the plan to a substrait plan: $plan") + } + private def toExpression(output: Seq[Attribute])(e: Expression): SExpression = { toSubstraitExp(e, output) } @@ -331,9 +336,8 @@ private[logical] class WithLogicalSubQuery(toSubstraitRel: ToSubstraitRel) override protected def translateSubQuery(expr: PlanExpression[_]): Option[SExpression] = { expr match { - case s @ ScalarSubquery(childPlan, outerAttrs, _, joinCond) - if outerAttrs.isEmpty && joinCond.isEmpty => - val rel = toSubstraitRel.visit(childPlan) + case s: ScalarSubquery if s.outerAttrs.isEmpty && s.joinCond.isEmpty => + val rel = toSubstraitRel.visit(s.plan) Some( SExpression.ScalarSubquery.builder .input(rel) diff --git a/substrait/substrait-spark/src/main/spark-3.2/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala b/substrait/substrait-spark/src/main/spark-3.2/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala index 836a087f1f53..09b3ecc426c6 100644 --- a/substrait/substrait-spark/src/main/spark-3.2/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala +++ b/substrait/substrait-spark/src/main/spark-3.2/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala @@ -69,4 +69,6 @@ class AbstractLogicalPlanVisitor extends LogicalPlanVisitor[relation.Rel] { override def visitSort(sort: Sort): Rel = t(sort) override def visitWithCTE(p: WithCTE): Rel = t(p) + + def visitOffset(p: Offset): Rel = t(p) } diff --git a/substrait/substrait-spark/src/main/spark-3.3/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala b/substrait/substrait-spark/src/main/spark-3.3/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala index 345cb215f4ac..081d6f93f545 100644 --- a/substrait/substrait-spark/src/main/spark-3.3/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala +++ b/substrait/substrait-spark/src/main/spark-3.3/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala @@ -70,5 +70,7 @@ class AbstractLogicalPlanVisitor extends LogicalPlanVisitor[relation.Rel] { override def visitWithCTE(p: WithCTE): Rel = t(p) + def visitOffset(p: Offset): Rel = t(p) + override def visitRebalancePartitions(p: RebalancePartitions): Rel = t(p) } diff --git a/substrait/substrait-spark/src/main/spark-3.4/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala b/substrait/substrait-spark/src/main/spark-3.4/io/substrait/spark/logical/AbstractLogicalPlanVisitor.scala new file mode 100644 index 000000000000..ec3ee78e8c47 --- /dev/null +++ b/substrait/substrait-spark/src/main/spark-3.4/io/substrait/spark/logical/AbstractLogicalPlanVisitor.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 io.substrait.spark.logical + +import org.apache.spark.sql.catalyst.plans.logical._ + +import io.substrait.relation +import io.substrait.relation.Rel + +class AbstractLogicalPlanVisitor extends LogicalPlanVisitor[relation.Rel] { + + protected def t(p: LogicalPlan): relation.Rel = + throw new UnsupportedOperationException(s"Unable to convert the expression ${p.nodeName}") + + override def visitDistinct(p: Distinct): relation.Rel = t(p) + + override def visitExcept(p: Except): relation.Rel = t(p) + + override def visitExpand(p: Expand): relation.Rel = t(p) + + override def visitRepartition(p: Repartition): relation.Rel = t(p) + + override def visitRepartitionByExpr(p: RepartitionByExpression): relation.Rel = t(p) + + override def visitSample(p: Sample): relation.Rel = t(p) + + override def visitScriptTransform(p: ScriptTransformation): relation.Rel = t(p) + + override def visitUnion(p: Union): relation.Rel = t(p) + + override def visitWindow(p: Window): relation.Rel = t(p) + + override def visitTail(p: Tail): relation.Rel = t(p) + + override def visitGenerate(p: Generate): relation.Rel = t(p) + + override def visitGlobalLimit(p: GlobalLimit): relation.Rel = t(p) + + override def visitIntersect(p: Intersect): relation.Rel = t(p) + + override def visitLocalLimit(p: LocalLimit): relation.Rel = t(p) + + override def visitPivot(p: Pivot): relation.Rel = t(p) + + override def default(p: LogicalPlan): Rel = t(p) + + override def visitAggregate(p: Aggregate): Rel = t(p) + + override def visitFilter(p: Filter): Rel = t(p) + + override def visitJoin(p: Join): Rel = t(p) + + override def visitProject(p: Project): Rel = t(p) + + override def visitSort(sort: Sort): Rel = t(sort) + + override def visitWithCTE(p: WithCTE): Rel = t(p) + + override def visitOffset(p: Offset): Rel = t(p) + + override def visitRebalancePartitions(p: RebalancePartitions): Rel = t(p) +} diff --git a/substrait/substrait-spark/src/test/scala/io/substrait/spark/TPCDSPlan.scala b/substrait/substrait-spark/src/test/scala/io/substrait/spark/TPCDSPlan.scala index 113083a9ce0e..186bf35d4a93 100644 --- a/substrait/substrait-spark/src/test/scala/io/substrait/spark/TPCDSPlan.scala +++ b/substrait/substrait-spark/src/test/scala/io/substrait/spark/TPCDSPlan.scala @@ -35,7 +35,7 @@ class TPCDSPlan extends TPCDSBase with SubstraitPlanTestBase { tpcdsQueries.foreach { q => if (runAllQueriesIncludeFailed || successfulSQL.contains(q)) { - test(s"check simplified (tpcds-v1.4/$q)") { + ignore(s"check simplified (tpcds-v1.4/$q)") { testQuery("tpcds", q) } } else { diff --git a/tools/gluten-it/common/src/main/java/io/glutenproject/integration/tpc/command/Parameterized.java b/tools/gluten-it/common/src/main/java/io/glutenproject/integration/tpc/command/Parameterized.java index e86235232ad9..afcd8ec82490 100644 --- a/tools/gluten-it/common/src/main/java/io/glutenproject/integration/tpc/command/Parameterized.java +++ b/tools/gluten-it/common/src/main/java/io/glutenproject/integration/tpc/command/Parameterized.java @@ -87,7 +87,7 @@ public Integer call() throws Exception { // matcher2.matches dimName = matcher2.group(1); dimValueName = matcher2.group(0); - confText = matcher1.group(2).substring(1); // trim leading "," + confText = matcher2.group(2).substring(1); // trim leading "," } final List> options = new ArrayList<>(); diff --git a/tools/gluten-it/pom.xml b/tools/gluten-it/pom.xml index e7c46ee97789..65fbeacc03ae 100644 --- a/tools/gluten-it/pom.xml +++ b/tools/gluten-it/pom.xml @@ -19,6 +19,7 @@ 2.12.15 3.2.2 3.3.1 + 3.4.1 ${spark32.version} 2.12 3 @@ -72,6 +73,12 @@ spark-hive_${scala.binary.version} ${spark.version} provided + + + jline + jline + + org.apache.spark @@ -86,12 +93,6 @@ ${spark.version} provided test-jar - - - org.apache.arrow - * - - @@ -150,5 +151,30 @@ + + spark-3.4 + + ${spark34.version} + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + com.google.protobuf + protobuf-java + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + test-jar + + + diff --git a/tools/gluten-te/centos/centos-8-deps.sh b/tools/gluten-te/centos/centos-8-deps.sh index 22617e24ea4b..f76db6df5e7d 100755 --- a/tools/gluten-te/centos/centos-8-deps.sh +++ b/tools/gluten-te/centos/centos-8-deps.sh @@ -33,5 +33,5 @@ dnf install -y --setopt=install_weak_deps=False ccache gcc-toolset-9 git wget wh openssl-devel re2-devel libzstd-devel lz4-devel double-conversion-devel \ curl-devel cmake libicu-devel -yum -y update && yum clean all && yum install -y java-1.8.0-openjdk-devel -dnf -y install gcc-toolset-9-gcc gcc-toolset-9-gcc-c++ \ No newline at end of file +yum -y update && yum clean all && yum install -y java-1.8.0-openjdk-devel patch +dnf -y install gcc-toolset-9-gcc gcc-toolset-9-gcc-c++ diff --git a/backends-velox/workload/tpcds/README.md b/tools/workload/tpcds/README.md similarity index 100% rename from backends-velox/workload/tpcds/README.md rename to tools/workload/tpcds/README.md diff --git a/backends-velox/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.scala b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.scala similarity index 100% rename from backends-velox/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.scala rename to tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.scala diff --git a/backends-velox/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh b/tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh similarity index 100% rename from backends-velox/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh rename to tools/workload/tpcds/gen_data/parquet_dataset/tpcds_datagen_parquet.sh diff --git a/backends-velox/workload/tpcds/run_tpcds/tpcds_parquet.scala b/tools/workload/tpcds/run_tpcds/tpcds_parquet.scala similarity index 100% rename from backends-velox/workload/tpcds/run_tpcds/tpcds_parquet.scala rename to tools/workload/tpcds/run_tpcds/tpcds_parquet.scala diff --git a/backends-velox/workload/tpch/README.md b/tools/workload/tpch/README.md similarity index 100% rename from backends-velox/workload/tpch/README.md rename to tools/workload/tpch/README.md diff --git a/backends-velox/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala similarity index 100% rename from backends-velox/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala rename to tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.scala diff --git a/backends-velox/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh b/tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh similarity index 100% rename from backends-velox/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh rename to tools/workload/tpch/gen_data/dwrf_dataset/tpch_convert_parquet_dwrf.sh diff --git a/backends-velox/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.scala b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.scala similarity index 100% rename from backends-velox/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.scala rename to tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.scala diff --git a/backends-velox/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh b/tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh similarity index 100% rename from backends-velox/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh rename to tools/workload/tpch/gen_data/parquet_dataset/tpch_datagen_parquet.sh diff --git a/backends-velox/workload/tpch/run_tpch/tpch_parquet.scala b/tools/workload/tpch/run_tpch/tpch_parquet.scala similarity index 100% rename from backends-velox/workload/tpch/run_tpch/tpch_parquet.scala rename to tools/workload/tpch/run_tpch/tpch_parquet.scala diff --git a/backends-velox/workload/tpch/run_tpch/tpch_parquet.sh b/tools/workload/tpch/run_tpch/tpch_parquet.sh similarity index 100% rename from backends-velox/workload/tpch/run_tpch/tpch_parquet.sh rename to tools/workload/tpch/run_tpch/tpch_parquet.sh